diff --git a/hadoop-mapreduce-project/CHANGES.txt b/hadoop-mapreduce-project/CHANGES.txt
index 3804cee4f0..e1742a94d6 100644
--- a/hadoop-mapreduce-project/CHANGES.txt
+++ b/hadoop-mapreduce-project/CHANGES.txt
@@ -546,6 +546,9 @@ Release 2.1.0-beta - UNRELEASED
MAPREDUCE-5304. mapreduce.Job killTask/failTask/getTaskCompletionEvents
methods have incompatible signature changes. (kkambatl via tucu)
+ MAPREDUCE-5298. Moved MapReduce services to YARN-530 stricter lifecycle.
+ (Steve Loughran via vinodkv)
+
Release 2.0.5-alpha - 06/06/2013
INCOMPATIBLE CHANGES
diff --git a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
index ecac4244c3..e4c1001a71 100644
--- a/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
+++ b/hadoop-mapreduce-project/dev-support/findbugs-exclude.xml
@@ -454,20 +454,32 @@
-
-
-
-
-
-
-
-
-
-
-
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
+
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
index d5243107ef..5fef8885b9 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/LocalContainerLauncher.java
@@ -18,10 +18,8 @@
package org.apache.hadoop.mapred;
-import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.IOException;
-import java.io.PrintStream;
import java.util.HashSet;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.LinkedBlockingQueue;
@@ -113,15 +111,17 @@ public LocalContainerLauncher(AppContext context,
// after running (e.g., "localizeForTask()" or "localizeForMapTask()").
}
- public void start() {
+ public void serviceStart() throws Exception {
eventHandlingThread = new Thread(new SubtaskRunner(), "uber-SubtaskRunner");
eventHandlingThread.start();
- super.start();
+ super.serviceStart();
}
- public void stop() {
- eventHandlingThread.interrupt();
- super.stop();
+ public void serviceStop() throws Exception {
+ if (eventHandlingThread != null) {
+ eventHandlingThread.interrupt();
+ }
+ super.serviceStop();
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
index e30d41401b..d3d1ee8a4c 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/TaskAttemptListenerImpl.java
@@ -95,17 +95,17 @@ public TaskAttemptListenerImpl(AppContext context,
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
registerHeartbeatHandler(conf);
commitWindowMs = conf.getLong(MRJobConfig.MR_AM_COMMIT_WINDOW_MS,
MRJobConfig.DEFAULT_MR_AM_COMMIT_WINDOW_MS);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
startRpcServer();
- super.start();
+ super.serviceStart();
}
protected void registerHeartbeatHandler(Configuration conf) {
@@ -144,13 +144,15 @@ void refreshServiceAcls(Configuration configuration,
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
stopRpcServer();
- super.stop();
+ super.serviceStop();
}
protected void stopRpcServer() {
- server.stop();
+ if (server != null) {
+ server.stop();
+ }
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java
index e4f6ffa17a..97b05a88fa 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryCopyService.java
@@ -18,7 +18,6 @@
package org.apache.hadoop.mapreduce.jobhistory;
-import java.io.FileNotFoundException;
import java.io.IOException;
import org.apache.commons.logging.Log;
@@ -58,8 +57,8 @@ public JobHistoryCopyService(ApplicationAttemptId applicationAttemptId,
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
}
@Override
@@ -71,14 +70,14 @@ public void handleEvent(HistoryEvent event) throws IOException {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
try {
//TODO should we parse on a background thread???
parse();
} catch (IOException e) {
throw new YarnRuntimeException(e);
}
- super.start();
+ super.serviceStart();
}
private void parse() throws IOException {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
index 6c671bf485..dfc716d4a5 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java
@@ -115,7 +115,7 @@ public JobHistoryEventHandler(AppContext context, int startCount) {
* Creates these directories if they do not already exist.
*/
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
String jobId =
TypeConverter.fromYarn(context.getApplicationID()).toString();
@@ -220,7 +220,7 @@ public void init(Configuration conf) {
MRJobConfig.MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD,
MRJobConfig.DEFAULT_MR_AM_HISTORY_USE_BATCHED_FLUSH_QUEUE_SIZE_THRESHOLD);
- super.init(conf);
+ super.serviceInit(conf);
}
private void mkdir(FileSystem fs, Path path, FsPermission fsp)
@@ -243,7 +243,7 @@ private void mkdir(FileSystem fs, Path path, FsPermission fsp)
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
eventHandlingThread = new Thread(new Runnable() {
@Override
public void run() {
@@ -275,38 +275,48 @@ public void run() {
boolean isInterrupted = Thread.interrupted();
handleEvent(event);
if (isInterrupted) {
- Thread.currentThread().interrupt();
+ LOG.debug("Event handling interrupted");
+ Thread.currentThread().interrupt();
+ }
}
}
}
- }
- });
+ }, "eventHandlingThread");
eventHandlingThread.start();
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
LOG.info("Stopping JobHistoryEventHandler. "
+ "Size of the outstanding queue size is " + eventQueue.size());
stopped = true;
//do not interrupt while event handling is in progress
synchronized(lock) {
- if (eventHandlingThread != null)
+ if (eventHandlingThread != null) {
+ LOG.debug("Interrupting Event Handling thread");
eventHandlingThread.interrupt();
+ } else {
+ LOG.debug("Null event handling thread");
+ }
}
try {
- if (eventHandlingThread != null)
+ if (eventHandlingThread != null) {
+ LOG.debug("Waiting for Event Handling thread to complete");
eventHandlingThread.join();
+ }
} catch (InterruptedException ie) {
- LOG.info("Interruped Exception while stopping", ie);
+ LOG.info("Interrupted Exception while stopping", ie);
}
// Cancel all timers - so that they aren't invoked during or after
// the metaInfo object is wrapped up.
for (MetaInfo mi : fileMap.values()) {
try {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Shutting down timer for " + mi);
+ }
mi.shutDownTimer();
} catch (IOException e) {
LOG.info("Exception while cancelling delayed flush timer. "
@@ -354,7 +364,7 @@ public void stop() {
}
}
LOG.info("Stopped JobHistoryEventHandler. super.stop()");
- super.stop();
+ super.serviceStop();
}
protected EventWriter createEventWriter(Path historyFilePath)
@@ -725,6 +735,7 @@ private class FlushTimerTask extends TimerTask {
@Override
public void run() {
+ LOG.debug("In flush timer task");
synchronized (lock) {
try {
if (!metaInfo.isTimerShutDown() && shouldRun)
@@ -790,7 +801,14 @@ boolean isTimerShutDown() {
return isTimerShutDown;
}
+ @Override
+ public String toString() {
+ return "Job MetaInfo for "+ jobSummary.getJobId()
+ + " history file " + historyFile;
+ }
+
void closeWriter() throws IOException {
+ LOG.debug("Closing Writer");
synchronized (lock) {
if (writer != null) {
writer.close();
@@ -800,6 +818,7 @@ void closeWriter() throws IOException {
}
void writeEvent(HistoryEvent event) throws IOException {
+ LOG.debug("Writing event");
synchronized (lock) {
if (writer != null) {
writer.write(event);
@@ -849,6 +868,9 @@ void maybeFlush(HistoryEvent historyEvent) throws IOException {
}
void flush() throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Flushing " + toString());
+ }
synchronized (lock) {
if (numUnflushedCompletionEvents != 0) { // skipped timer cancel.
writer.flush();
@@ -859,6 +881,9 @@ void flush() throws IOException {
}
void shutDownTimer() throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Shutting down timer "+ toString());
+ }
synchronized (lock) {
isTimerShutDown = true;
flushTimer.cancel();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
index eb4e6c3eef..43e7e19a23 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
@@ -128,6 +128,7 @@
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.hadoop.yarn.service.CompositeService;
import org.apache.hadoop.yarn.service.Service;
+import org.apache.hadoop.yarn.service.ServiceOperations;
import org.apache.hadoop.yarn.util.ConverterUtils;
import com.google.common.annotations.VisibleForTesting;
@@ -227,7 +228,7 @@ public MRAppMaster(ApplicationAttemptId applicationAttemptId,
}
@Override
- public void init(final Configuration conf) {
+ protected void serviceInit(final Configuration conf) throws Exception {
conf.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
downloadTokensAndSetupUGI(conf);
@@ -416,7 +417,7 @@ public void init(final Configuration conf) {
addIfService(historyService);
}
- super.init(conf);
+ super.serviceInit(conf);
} // end of init()
protected Dispatcher createDispatcher() {
@@ -784,7 +785,7 @@ private final class ContainerAllocatorRouter extends AbstractService
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
if (job.isUber()) {
this.containerAllocator = new LocalContainerAllocator(
this.clientService, this.context, nmHost, nmPort, nmHttpPort
@@ -795,13 +796,13 @@ public synchronized void start() {
}
((Service)this.containerAllocator).init(getConfig());
((Service)this.containerAllocator).start();
- super.start();
+ super.serviceStart();
}
@Override
- public synchronized void stop() {
- ((Service)this.containerAllocator).stop();
- super.stop();
+ protected void serviceStop() throws Exception {
+ ServiceOperations.stop((Service) this.containerAllocator);
+ super.serviceStop();
}
@Override
@@ -843,7 +844,7 @@ private final class ContainerLauncherRouter extends AbstractService
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
if (job.isUber()) {
this.containerLauncher = new LocalContainerLauncher(context,
(TaskUmbilicalProtocol) taskAttemptListener);
@@ -852,7 +853,7 @@ public synchronized void start() {
}
((Service)this.containerLauncher).init(getConfig());
((Service)this.containerLauncher).start();
- super.start();
+ super.serviceStart();
}
@Override
@@ -861,9 +862,9 @@ public void handle(ContainerLauncherEvent event) {
}
@Override
- public synchronized void stop() {
- ((Service)this.containerLauncher).stop();
- super.stop();
+ protected void serviceStop() throws Exception {
+ ServiceOperations.stop((Service) this.containerLauncher);
+ super.serviceStop();
}
}
@@ -873,7 +874,7 @@ private final class StagingDirCleaningService extends AbstractService {
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
try {
if(isLastAMRetry) {
cleanupStagingDir();
@@ -884,7 +885,7 @@ public synchronized void stop() {
} catch (IOException io) {
LOG.error("Failed to cleanup staging dir: ", io);
}
- super.stop();
+ super.serviceStop();
}
}
@@ -951,7 +952,7 @@ public ClusterInfo getClusterInfo() {
@SuppressWarnings("unchecked")
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
amInfos = new LinkedList();
completedTasksFromPreviousRun = new HashMap();
@@ -1011,7 +1012,7 @@ public void start() {
}
//start all the components
- super.start();
+ super.serviceStart();
// All components have started, start the job.
startJobs();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
index 153b5a8d6d..2ac647bbff 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/TaskHeartbeatHandler.java
@@ -85,26 +85,28 @@ public TaskHeartbeatHandler(EventHandler eventHandler, Clock clock,
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
taskTimeOut = conf.getInt(MRJobConfig.TASK_TIMEOUT, 5 * 60 * 1000);
taskTimeOutCheckInterval =
conf.getInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 30 * 1000);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
lostTaskCheckerThread = new Thread(new PingChecker());
lostTaskCheckerThread.setName("TaskHeartbeatHandler PingChecker");
lostTaskCheckerThread.start();
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
stopped = true;
- lostTaskCheckerThread.interrupt();
- super.stop();
+ if (lostTaskCheckerThread != null) {
+ lostTaskCheckerThread.interrupt();
+ }
+ super.serviceStop();
}
public void progressing(TaskAttemptId attemptID) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
index e0a2fd7c09..9fff2135e8 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
@@ -112,7 +112,7 @@ public MRClientService(AppContext appContext) {
this.protocolHandler = new MRClientProtocolHandler();
}
- public void start() {
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
InetSocketAddress address = new InetSocketAddress(0);
@@ -150,7 +150,7 @@ public void start() {
} catch (Exception e) {
LOG.error("Webapps failed to start. Ignoring for now:", e);
}
- super.start();
+ super.serviceStart();
}
void refreshServiceAcls(Configuration configuration,
@@ -158,12 +158,15 @@ void refreshServiceAcls(Configuration configuration,
this.server.refreshServiceAcl(configuration, policyProvider);
}
- public void stop() {
- server.stop();
+ @Override
+ protected void serviceStop() throws Exception {
+ if (server != null) {
+ server.stop();
+ }
if (webApp != null) {
webApp.stop();
}
- super.stop();
+ super.serviceStop();
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/commit/CommitterEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/commit/CommitterEventHandler.java
index 123b1f9162..d3e47f05d4 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/commit/CommitterEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/commit/CommitterEventHandler.java
@@ -87,8 +87,8 @@ public CommitterEventHandler(AppContext context, OutputCommitter committer,
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
commitThreadCancelTimeoutMs = conf.getInt(
MRJobConfig.MR_AM_COMMITTER_CANCEL_TIMEOUT_MS,
MRJobConfig.DEFAULT_MR_AM_COMMITTER_CANCEL_TIMEOUT_MS);
@@ -108,7 +108,7 @@ public void init(Configuration conf) {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
ThreadFactory tf = new ThreadFactoryBuilder()
.setNameFormat("CommitterEvent Processor #%d")
.build();
@@ -134,7 +134,7 @@ public void run() {
});
eventHandlingThread.setName("CommitterEvent Handler");
eventHandlingThread.start();
- super.start();
+ super.serviceStart();
}
@@ -148,14 +148,18 @@ public void handle(CommitterEvent event) {
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (stopped.getAndSet(true)) {
// return if already stopped
return;
}
- eventHandlingThread.interrupt();
- launcherPool.shutdown();
- super.stop();
+ if (eventHandlingThread != null) {
+ eventHandlingThread.interrupt();
+ }
+ if (launcherPool != null) {
+ launcherPool.shutdown();
+ }
+ super.serviceStop();
}
private synchronized void jobCommitStarted() throws IOException {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
index 25b0f85b1a..8503f5ecbb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
@@ -239,7 +239,7 @@ public ContainerLauncherImpl(AppContext context) {
}
@Override
- public synchronized void init(Configuration config) {
+ protected void serviceInit(Configuration config) throws Exception {
Configuration conf = new Configuration(config);
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
@@ -249,14 +249,14 @@ public synchronized void init(Configuration config) {
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize);
this.rpc = createYarnRPC(conf);
- super.init(conf);
+ super.serviceInit(conf);
}
protected YarnRPC createYarnRPC(Configuration conf) {
return YarnRPC.create(conf);
}
- public void start() {
+ protected void serviceStart() throws Exception {
ThreadFactory tf = new ThreadFactoryBuilder().setNameFormat(
"ContainerLauncher #%d").setDaemon(true).build();
@@ -317,7 +317,7 @@ public void run() {
};
eventHandlingThread.setName("ContainerLauncher Event Handler");
eventHandlingThread.start();
- super.start();
+ super.serviceStart();
}
private void shutdownAllContainers() {
@@ -328,16 +328,20 @@ private void shutdownAllContainers() {
}
}
- public void stop() {
+ protected void serviceStop() throws Exception {
if (stopped.getAndSet(true)) {
// return if already stopped
return;
}
// shutdown any containers that might be left running
shutdownAllContainers();
- eventHandlingThread.interrupt();
- launcherPool.shutdownNow();
- super.stop();
+ if (eventHandlingThread != null) {
+ eventHandlingThread.interrupt();
+ }
+ if (launcherPool != null) {
+ launcherPool.shutdownNow();
+ }
+ super.serviceStop();
}
protected EventProcessor createEventProcessor(ContainerLauncherEvent event) {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
index aed2ad9470..8ae8297beb 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
@@ -80,8 +80,8 @@ public LocalContainerAllocator(ClientService clientService,
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
retryInterval =
getConfig().getLong(MRJobConfig.MR_AM_TO_RM_WAIT_INTERVAL_MS,
MRJobConfig.DEFAULT_MR_AM_TO_RM_WAIT_INTERVAL_MS);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
index 369b5255ae..b04291a0c7 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
@@ -99,22 +99,22 @@ public RMCommunicator(ClientService clientService, AppContext context) {
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
rmPollInterval =
conf.getInt(MRJobConfig.MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS,
MRJobConfig.DEFAULT_MR_AM_TO_RM_HEARTBEAT_INTERVAL_MS);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
scheduler= createSchedulerProxy();
register();
startAllocatorThread();
JobID id = TypeConverter.fromYarn(this.applicationId);
JobId jobId = TypeConverter.toYarn(id);
job = context.getJob(jobId);
- super.start();
+ super.serviceStart();
}
protected AppContext getContext() {
@@ -211,21 +211,23 @@ protected Resource getMaxContainerCapability() {
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (stopped.getAndSet(true)) {
// return if already stopped
return;
}
- allocatorThread.interrupt();
- try {
- allocatorThread.join();
- } catch (InterruptedException ie) {
- LOG.warn("InterruptedException while stopping", ie);
+ if (allocatorThread != null) {
+ allocatorThread.interrupt();
+ try {
+ allocatorThread.join();
+ } catch (InterruptedException ie) {
+ LOG.warn("InterruptedException while stopping", ie);
+ }
}
if(shouldUnregister) {
unregister();
}
- super.stop();
+ super.serviceStop();
}
protected void startAllocatorThread() {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
index 106f2f59f8..6a8d8fda9b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerAllocator.java
@@ -156,8 +156,8 @@ public RMContainerAllocator(ClientService clientService, AppContext context) {
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
reduceSlowStart = conf.getFloat(
MRJobConfig.COMPLETED_MAPS_FOR_REDUCE_SLOWSTART,
DEFAULT_COMPLETED_MAPS_PERCENT_FOR_REDUCE_SLOWSTART);
@@ -176,7 +176,7 @@ public void init(Configuration conf) {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
this.eventHandlingThread = new Thread() {
@SuppressWarnings("unchecked")
@Override
@@ -208,7 +208,7 @@ public void run() {
}
};
this.eventHandlingThread.start();
- super.start();
+ super.serviceStart();
}
@Override
@@ -242,13 +242,15 @@ protected synchronized void heartbeat() throws Exception {
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (stopped.getAndSet(true)) {
// return if already stopped
return;
}
- eventHandlingThread.interrupt();
- super.stop();
+ if (eventHandlingThread != null) {
+ eventHandlingThread.interrupt();
+ }
+ super.serviceStop();
scheduleStats.log("Final Stats: ");
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
index fff8415d2d..8f43648f16 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMContainerRequestor.java
@@ -124,8 +124,8 @@ public String toString() {
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
nodeBlacklistingEnabled =
conf.getBoolean(MRJobConfig.MR_AM_JOB_NODE_BLACKLISTING_ENABLE, true);
LOG.info("nodeBlacklistingEnabled:" + nodeBlacklistingEnabled);
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java
index 3ecb94e493..235e304bda 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/speculate/DefaultSpeculator.java
@@ -166,7 +166,7 @@ public DefaultSpeculator(Configuration conf, AppContext context, Clock clock) {
// looking for speculation opportunities
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
Runnable speculationBackgroundCore
= new Runnable() {
@Override
@@ -202,17 +202,17 @@ public void run() {
(speculationBackgroundCore, "DefaultSpeculator background processing");
speculationBackgroundThread.start();
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
- stopped = true;
+ protected void serviceStop()throws Exception {
+ stopped = true;
// this could be called before background thread is established
if (speculationBackgroundThread != null) {
speculationBackgroundThread.interrupt();
}
- super.stop();
+ super.serviceStop();
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
index 422d8cd5f2..2de5ab1f0f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
@@ -350,7 +350,7 @@ public JHEvenHandlerForTest(AppContext context, int startCount) {
}
@Override
- public void start() {
+ protected void serviceStart() {
}
@Override
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
index 96854a6ecb..6e24137795 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
@@ -207,7 +207,7 @@ public MRApp(ApplicationAttemptId appAttemptId, ContainerId amContainerId,
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
try {
//Create the staging directory if it does not exist
String user = UserGroupInformation.getCurrentUser().getShortUserName();
@@ -218,7 +218,7 @@ public void init(Configuration conf) {
throw new YarnRuntimeException("Error creating staging dir", e);
}
- super.init(conf);
+ super.serviceInit(conf);
if (this.clusterInfo != null) {
getContext().getClusterInfo().setMinContainerCapability(
this.clusterInfo.getMinContainerCapability());
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
index 4b6b0e39e7..0286b30dce 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
@@ -130,7 +130,7 @@ public void handle(ContainerAllocatorEvent event) {
}
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
thread = new Thread(new Runnable() {
@Override
public void run() {
@@ -168,13 +168,15 @@ public void run() {
}
});
thread.start();
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
- thread.interrupt();
- super.stop();
+ protected void serviceStop() throws Exception {
+ if (thread != null) {
+ thread.interrupt();
+ }
+ super.serviceStop();
}
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
index 05111f926e..2eddbbf4ec 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestFail.java
@@ -261,10 +261,11 @@ protected TaskAttemptListener createTaskAttemptListener(AppContext context) {
public InetSocketAddress getAddress() {
return NetUtils.createSocketAddr("localhost", 1234);
}
- public void init(Configuration conf) {
+
+ protected void serviceInit(Configuration conf) throws Exception {
conf.setInt(MRJobConfig.TASK_TIMEOUT, 1*1000);//reduce timeout
conf.setInt(MRJobConfig.TASK_TIMEOUT_CHECK_INTERVAL_MS, 1*1000);
- super.init(conf);
+ super.serviceInit(conf);
}
};
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
index 0153b95fc0..8b4448ab47 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
@@ -438,9 +438,9 @@ public MRAppMasterTest(ApplicationAttemptId applicationAttemptId,
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
if (!overrideInit) {
- super.init(conf);
+ super.serviceInit(conf);
}
this.conf = conf;
}
@@ -472,7 +472,7 @@ protected RMHeartbeatHandler getRMHeartbeatHandler() {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
if (overrideStart) {
try {
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
@@ -483,7 +483,7 @@ public void start() {
fail(e.getMessage());
}
} else {
- super.start();
+ super.serviceStart();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
index a8cedc6176..5648bd8f5f 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
@@ -56,6 +56,7 @@
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.service.Service;
import org.junit.Test;
@@ -185,10 +186,13 @@ public void testDeletionofStagingOnKillLastTry() throws IOException {
ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc, 1); //no retry
appMaster.init(conf);
+ assertTrue("appMaster.isLastAMRetry() is false", appMaster.isLastAMRetry());
//simulate the process being killed
MRAppMaster.MRAppMasterShutdownHook hook =
new MRAppMaster.MRAppMasterShutdownHook(appMaster);
hook.run();
+ assertTrue("MRAppMaster isn't stopped",
+ appMaster.isInState(Service.STATE.STOPPED));
verify(fs).delete(stagingJobPath, true);
}
@@ -240,8 +244,8 @@ protected Job createJob(Configuration conf, JobStateInternal forcedState,
}
@Override
- public void start() {
- super.start();
+ public void serviceStart() throws Exception {
+ super.serviceStart();
DefaultMetricsSystem.shutdown();
}
@@ -329,9 +333,9 @@ public void handle(ContainerAllocatorEvent event) {
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
stoppedContainerAllocator = true;
- super.stop();
+ super.serviceStop();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
index 3a0e8f4b49..a557823c24 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/CachedHistoryStorage.java
@@ -60,7 +60,8 @@ public void setHistoryFileManager(HistoryFileManager hsManager) {
@SuppressWarnings("serial")
@Override
- public void init(Configuration conf) throws YarnRuntimeException {
+ public void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
LOG.info("CachedHistoryStorage Init");
loadedJobCacheSize = conf.getInt(
@@ -74,8 +75,6 @@ public boolean removeEldestEntry(final Map.Entry eldest) {
return super.size() > loadedJobCacheSize;
}
});
-
- super.init(conf);
}
public CachedHistoryStorage() {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
index 4fa0df54a1..2d936c6a93 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryClientService.java
@@ -111,7 +111,7 @@ public HistoryClientService(HistoryContext history,
this.jhsDTSecretManager = jhsDTSecretManager;
}
- public void start() {
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
initializeWebApp(conf);
@@ -138,7 +138,7 @@ public void start() {
server.getListenerAddress());
LOG.info("Instantiated MRClientService at " + this.bindAddress);
- super.start();
+ super.serviceStart();
}
private void initializeWebApp(Configuration conf) {
@@ -155,14 +155,14 @@ private void initializeWebApp(Configuration conf) {
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (server != null) {
server.stop();
}
if (webApp != null) {
webApp.stop();
}
- super.stop();
+ super.serviceStop();
}
@Private
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
index eb21e4a733..3f5af94ea1 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/HistoryFileManager.java
@@ -244,6 +244,9 @@ public HistoryFileInfo addIfAbsent(HistoryFileInfo fileInfo) {
}
public void delete(HistoryFileInfo fileInfo) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Removing from cache " + fileInfo);
+ }
cache.remove(fileInfo.getJobId());
}
@@ -275,6 +278,10 @@ public synchronized void scanIfNeeded(FileStatus fs) {
} catch (IOException e) {
LOG.error("Error while trying to scan the directory " + p, e);
}
+ } else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Scan not needed of " + fs.getPath());
+ }
}
}
}
@@ -314,9 +321,21 @@ public synchronized boolean isDeleted() {
return state == HistoryInfoState.DELETED;
}
+ @Override
+ public String toString() {
+ return "HistoryFileInfo jobID " + getJobId()
+ + " historyFile = " + historyFile;
+ }
+
private synchronized void moveToDone() throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("moveToDone: " + historyFile);
+ }
if (!isMovePending()) {
// It was either deleted or is already in done. Either way do nothing
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Move no longer pending");
+ }
return;
}
try {
@@ -398,6 +417,9 @@ synchronized Path getHistoryFile() {
}
private synchronized void delete() throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("deleting " + historyFile + " and " + confFile);
+ }
state = HistoryInfoState.DELETED;
doneDirFc.delete(doneDirFc.makeQualified(historyFile), false);
doneDirFc.delete(doneDirFc.makeQualified(confFile), false);
@@ -458,7 +480,7 @@ public HistoryFileManager() {
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
this.conf = conf;
int serialNumberLowDigits = 3;
@@ -519,7 +541,7 @@ public void init(Configuration conf) {
moveToDoneExecutor = new ThreadPoolExecutor(numMoveThreads, numMoveThreads,
1, TimeUnit.HOURS, new LinkedBlockingQueue(), tf);
- super.init(conf);
+ super.serviceInit(conf);
}
private void mkdir(FileContext fc, Path path, FsPermission fsp)
@@ -665,7 +687,7 @@ void scanIntermediateDirectory() throws IOException {
// case where we are looking for a particular job.
List userDirList = JobHistoryUtils.localGlobber(
intermediateDoneDirFc, intermediateDoneDirPath, "");
-
+ LOG.debug("Scanning intermediate dirs");
for (FileStatus userDir : userDirList) {
String name = userDir.getPath().getName();
UserLogDir dir = userDirModificationTimeMap.get(name);
@@ -687,9 +709,18 @@ void scanIntermediateDirectory() throws IOException {
* @throws IOException
*/
private void scanIntermediateDirectory(final Path absPath) throws IOException {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Scanning intermediate dir " + absPath);
+ }
List fileStatusList = scanDirectoryForHistoryFiles(absPath,
intermediateDoneDirFc);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Found " + fileStatusList.size() + " files");
+ }
for (FileStatus fs : fileStatusList) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("scanning file: "+ fs.getPath());
+ }
JobIndexInfo jobIndexInfo = FileNameIndexUtils.getIndexInfo(fs.getPath()
.getName());
String confFileName = JobHistoryUtils
@@ -711,6 +742,9 @@ private void scanIntermediateDirectory(final Path absPath) throws IOException {
LOG.warn("Error cleaning up a HistoryFile that is out of date.", e);
}
} else {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Scheduling move to done of " +found);
+ }
moveToDoneExecutor.execute(new Runnable() {
@Override
public void run() {
@@ -725,6 +759,9 @@ public void run() {
}
} else if (old != null && !old.isMovePending()) {
//This is a duplicate so just delete it
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Duplicate: deleting");
+ }
fileInfo.delete();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
index 4ad42ad8d5..03671e9107 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
@@ -71,7 +71,7 @@ public class JobHistory extends AbstractService implements HistoryContext {
private HistoryFileManager hsManager = null;
@Override
- public void init(Configuration conf) throws YarnRuntimeException {
+ protected void serviceInit(Configuration conf) throws Exception {
LOG.info("JobHistory Init");
this.conf = conf;
this.appID = ApplicationId.newInstance(0, 0);
@@ -98,11 +98,11 @@ public void init(Configuration conf) throws YarnRuntimeException {
}
storage.setHistoryFileManager(hsManager);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
hsManager.start();
if (storage instanceof Service) {
((Service) storage).start();
@@ -126,11 +126,11 @@ public void start() {
.scheduleAtFixedRate(new HistoryCleaner(),
30 * 1000l, runInterval, TimeUnit.MILLISECONDS);
}
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
LOG.info("Stopping JobHistory");
if (scheduledExecutor != null) {
LOG.info("Stopping History Cleaner/Move To Done");
@@ -151,11 +151,13 @@ public void stop() {
scheduledExecutor.shutdownNow();
}
}
- if (storage instanceof Service) {
+ if (storage != null && storage instanceof Service) {
((Service) storage).stop();
}
- hsManager.stop();
- super.stop();
+ if (hsManager != null) {
+ hsManager.stop();
+ }
+ super.serviceStop();
}
public JobHistory() {
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
index 44eb217888..c583fa4298 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistoryServer.java
@@ -65,7 +65,7 @@ public JobHistoryServer() {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
Configuration config = new YarnConfiguration(conf);
config.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
@@ -84,7 +84,7 @@ public synchronized void init(Configuration conf) {
addService(jobHistoryService);
addService(clientService);
addService(aggLogDelService);
- super.init(config);
+ super.serviceInit(config);
}
protected JHSDelegationTokenSecretManager createJHSSecretManager(
@@ -109,23 +109,25 @@ protected void doSecureLogin(Configuration conf) throws IOException {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
DefaultMetricsSystem.initialize("JobHistoryServer");
JvmMetrics.initSingleton("JobHistoryServer", null);
try {
jhsDTSecretManager.startThreads();
} catch(IOException io) {
LOG.error("Error while starting the Secret Manager threads", io);
- throw new RuntimeException(io);
+ throw io;
}
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
- jhsDTSecretManager.stopThreads();
+ protected void serviceStop() throws Exception {
+ if (jhsDTSecretManager != null) {
+ jhsDTSecretManager.stopThreads();
+ }
DefaultMetricsSystem.shutdown();
- super.stop();
+ super.serviceStop();
}
@Private
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java
index b5f17dbe67..670f463ffc 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryEvents.java
@@ -206,7 +206,7 @@ protected EventHandler createJobHistoryHandler(
AppContext context) {
return new JobHistoryEventHandler(context, getStartCount()) {
@Override
- public void start() {
+ protected void serviceStart() {
// Don't start any event draining thread.
super.eventHandlingThread = new Thread();
super.eventHandlingThread.start();
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
index faf3aebd25..65d356da99 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/TestJobHistoryServer.java
@@ -76,12 +76,15 @@ public void testStartStopServer() throws Exception {
historyServer.init(config);
assertEquals(STATE.INITED, historyServer.getServiceState());
assertEquals(3, historyServer.getServices().size());
+ HistoryClientService historyService = historyServer.getClientService();
+ assertNotNull(historyServer.getClientService());
+ assertEquals(STATE.INITED, historyService.getServiceState());
+
historyServer.start();
assertEquals(STATE.STARTED, historyServer.getServiceState());
+ assertEquals(STATE.STARTED, historyService.getServiceState());
historyServer.stop();
assertEquals(STATE.STOPPED, historyServer.getServiceState());
- assertNotNull(historyServer.getClientService());
- HistoryClientService historyService = historyServer.getClientService();
assertNotNull(historyService.getClientHandler().getConnectAddress());
@@ -202,7 +205,7 @@ public void testMainMethod() throws Exception {
@After
public void stop(){
- if(historyServer !=null && !STATE.STOPPED.equals(historyServer.getServiceState())){
+ if(historyServer != null) {
historyServer.stop();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
index e95c96c693..cccaf215ae 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestClientRedirect.java
@@ -233,7 +233,7 @@ public RMService(String name) {
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
clientServiceBindAddress = RMADDRESS;
/*
clientServiceBindAddress = conf.get(
@@ -241,11 +241,11 @@ public void init(Configuration conf) {
YarnConfiguration.DEFAULT_APPSMANAGER_BIND_ADDRESS);
*/
clientBindAddress = NetUtils.createSocketAddr(clientServiceBindAddress);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
// All the clients to appsManager are supposed to be authenticated via
// Kerberos if security is enabled, so no secretManager.
YarnRPC rpc = YarnRPC.create(getConfig());
@@ -253,7 +253,7 @@ public void start() {
this.server = rpc.getServer(ClientRMProtocol.class, this,
clientBindAddress, clientServerConf, null, 1);
this.server.start();
- super.start();
+ super.serviceStart();
}
@Override
@@ -416,9 +416,12 @@ public void start(Configuration conf) {
amRunning = true;
}
- public void stop() {
- server.stop();
- super.stop();
+ @Override
+ protected void serviceStop() throws Exception {
+ if (server != null) {
+ server.stop();
+ }
+ super.serviceStop();
amRunning = false;
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
index 6b60b36d35..57b09c57db 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
@@ -67,7 +67,7 @@ public void testGetRootQueues() throws IOException, InterruptedException {
ResourceMgrDelegate delegate = new ResourceMgrDelegate(
new YarnConfiguration()) {
@Override
- public synchronized void start() {
+ protected void serviceStart() {
this.rmClient = applicationsManager;
}
};
@@ -110,7 +110,7 @@ public void tesAllJobs() throws Exception {
ResourceMgrDelegate resourceMgrDelegate = new ResourceMgrDelegate(
new YarnConfiguration()) {
@Override
- public synchronized void start() {
+ protected void serviceStart() {
this.rmClient = applicationsManager;
}
};
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
index 7d6cf72354..71e3289af6 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
@@ -200,7 +200,7 @@ public void testResourceMgrDelegate() throws Exception {
final ClientRMProtocol clientRMProtocol = mock(ClientRMProtocol.class);
ResourceMgrDelegate delegate = new ResourceMgrDelegate(conf) {
@Override
- public synchronized void start() {
+ protected void serviceStart() {
this.rmClient = clientRMProtocol;
}
};
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java
index cc23475fbe..43dd22778b 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/TestYarnClientProtocolProvider.java
@@ -110,7 +110,7 @@ public void testClusterGetDelegationToken() throws Exception {
ResourceMgrDelegate rmgrDelegate = new ResourceMgrDelegate(
new YarnConfiguration(conf)) {
@Override
- public synchronized void start() {
+ protected void serviceStart() {
this.rmClient = cRMProtocol;
}
};
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
index e1cf3d4046..18bd662572 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/MiniMRYarnCluster.java
@@ -67,7 +67,7 @@ public MiniMRYarnCluster(String testName, int noOfNMs) {
}
@Override
- public void init(Configuration conf) {
+ public void serviceInit(Configuration conf) throws Exception {
conf.set(MRConfig.FRAMEWORK_NAME, MRConfig.YARN_FRAMEWORK_NAME);
if (conf.get(MRJobConfig.MR_AM_STAGING_DIR) == null) {
conf.set(MRJobConfig.MR_AM_STAGING_DIR, new File(getTestWorkDir(),
@@ -121,7 +121,7 @@ public void init(Configuration conf) {
// for corresponding uberized tests.
conf.setBoolean(MRJobConfig.JOB_UBERTASK_ENABLE, false);
- super.init(conf);
+ super.serviceInit(conf);
}
private class JobHistoryServerWrapper extends AbstractService {
@@ -130,7 +130,7 @@ public JobHistoryServerWrapper() {
}
@Override
- public synchronized void start() {
+ public synchronized void serviceStart() throws Exception {
try {
if (!getConfig().getBoolean(
JHAdminConfig.MR_HISTORY_MINICLUSTER_FIXED_PORTS,
@@ -156,7 +156,7 @@ public void run() {
if (historyServer.getServiceState() != STATE.STARTED) {
throw new IOException("HistoryServer failed to start");
}
- super.start();
+ super.serviceStart();
} catch (Throwable t) {
throw new YarnRuntimeException(t);
}
@@ -177,11 +177,11 @@ public void run() {
}
@Override
- public synchronized void stop() {
+ public synchronized void serviceStop() throws Exception {
if (historyServer != null) {
historyServer.stop();
}
- super.stop();
+ super.serviceStop();
}
}
diff --git a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
index b9af3af8dc..67a05114a2 100644
--- a/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
+++ b/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
@@ -267,7 +267,7 @@ public void stopApp(ApplicationId appId) {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
manageOsCache = conf.getBoolean(SHUFFLE_MANAGE_OS_CACHE,
DEFAULT_SHUFFLE_MANAGE_OS_CACHE);
@@ -287,12 +287,12 @@ public synchronized void init(Configuration conf) {
selector = new NioServerSocketChannelFactory(
Executors.newCachedThreadPool(bossFactory),
Executors.newCachedThreadPool(workerFactory));
- super.init(new Configuration(conf));
+ super.serviceInit(new Configuration(conf));
}
// TODO change AbstractService to throw InterruptedException
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
ServerBootstrap bootstrap = new ServerBootstrap(selector);
try {
@@ -308,19 +308,23 @@ public synchronized void start() {
conf.set(SHUFFLE_PORT_CONFIG_KEY, Integer.toString(port));
pipelineFact.SHUFFLE.setPort(port);
LOG.info(getName() + " listening on port " + port);
- super.start();
+ super.serviceStart();
sslFileBufferSize = conf.getInt(SUFFLE_SSL_FILE_BUFFER_SIZE_KEY,
DEFAULT_SUFFLE_SSL_FILE_BUFFER_SIZE);
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
accepted.close().awaitUninterruptibly(10, TimeUnit.SECONDS);
- ServerBootstrap bootstrap = new ServerBootstrap(selector);
- bootstrap.releaseExternalResources();
- pipelineFact.destroy();
- super.stop();
+ if (selector != null) {
+ ServerBootstrap bootstrap = new ServerBootstrap(selector);
+ bootstrap.releaseExternalResources();
+ }
+ if (pipelineFact != null) {
+ pipelineFact.destroy();
+ }
+ super.serviceStop();
}
@Override
diff --git a/hadoop-yarn-project/CHANGES.txt b/hadoop-yarn-project/CHANGES.txt
index 29a2bba333..89ba261eea 100644
--- a/hadoop-yarn-project/CHANGES.txt
+++ b/hadoop-yarn-project/CHANGES.txt
@@ -132,6 +132,10 @@ Release 2.1.0-beta - UNRELEASED
YARN-642. Removed health parameter from ResourceManager /nodes web-service
and cleaned the behaviour of the status parameter. (Sandy Ryza vid vinodkv)
+ YARN-530. Defined Service model strictly, implemented AbstractService for
+ robust subclassing and migrated yarn-common services. (Steve Loughran via
+ vinodkv)
+
NEW FEATURES
YARN-482. FS: Extend SchedulingMode to intermediate queues.
@@ -502,6 +506,9 @@ Release 2.1.0-beta - UNRELEASED
YARN-700. TestInfoBlock fails on Windows because of line ending missmatch.
(Ivan Mitic via cnauroth)
+ YARN-117. Migrated rest of YARN to the new service model. (Steve Louhran via
+ vinodkv)
+
BREAKDOWN OF HADOOP-8562 SUBTASKS AND RELATED JIRAS
YARN-158. Yarn creating package-info.java must not depend on sh.
diff --git a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
index fd7da24b03..77d2458af6 100644
--- a/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
+++ b/hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
@@ -24,6 +24,11 @@
+
+
+
+
+
@@ -172,6 +177,11 @@
+
+
+
+
+
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
index 43907f0966..87267054f8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/test/java/org/apache/hadoop/yarn/applications/distributedshell/TestDistributedShell.java
@@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.applications.distributedshell;
+import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
@@ -63,9 +64,16 @@ public static void setup() throws InterruptedException, IOException {
if (url == null) {
throw new RuntimeException("Could not find 'yarn-site.xml' dummy file in classpath");
}
- yarnCluster.getConfig().set("yarn.application.classpath", new File(url.getPath()).getParent());
+ Configuration yarnClusterConfig = yarnCluster.getConfig();
+ yarnClusterConfig.set("yarn.application.classpath", new File(url.getPath()).getParent());
+ //write the document to a buffer (not directly to the file, as that
+ //can cause the file being written to get read -which will then fail.
+ ByteArrayOutputStream bytesOut = new ByteArrayOutputStream();
+ yarnClusterConfig.writeXml(bytesOut);
+ bytesOut.close();
+ //write the bytes to the file in the classpath
OutputStream os = new FileOutputStream(new File(url.getPath()));
- yarnCluster.getConfig().writeXml(os);
+ os.write(bytesOut.toByteArray());
os.close();
}
try {
@@ -78,8 +86,11 @@ public static void setup() throws InterruptedException, IOException {
@AfterClass
public static void tearDown() throws IOException {
if (yarnCluster != null) {
- yarnCluster.stop();
- yarnCluster = null;
+ try {
+ yarnCluster.stop();
+ } finally {
+ yarnCluster = null;
+ }
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java
index 6ab474d43b..9ae5807789 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/TestUnmanagedAMLauncher.java
@@ -18,8 +18,10 @@
package org.apache.hadoop.yarn.applications.unmanagedamlauncher;
+import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
+import java.io.ByteArrayOutputStream;
import java.io.File;
import java.io.FileOutputStream;
import java.io.IOException;
@@ -54,14 +56,30 @@ public static void setup() throws InterruptedException, IOException {
TestUnmanagedAMLauncher.class.getSimpleName(), 1, 1, 1);
yarnCluster.init(conf);
yarnCluster.start();
+ //get the address
+ Configuration yarnClusterConfig = yarnCluster.getConfig();
+ LOG.info("MiniYARN ResourceManager published address: " +
+ yarnClusterConfig.get(YarnConfiguration.RM_ADDRESS));
+ LOG.info("MiniYARN ResourceManager published web address: " +
+ yarnClusterConfig.get(YarnConfiguration.RM_WEBAPP_ADDRESS));
+ String webapp = yarnClusterConfig.get(YarnConfiguration.RM_WEBAPP_ADDRESS);
+ assertTrue("Web app address still unbound to a host at " + webapp,
+ !webapp.startsWith("0.0.0.0"));
+ LOG.info("Yarn webapp is at "+ webapp);
URL url = Thread.currentThread().getContextClassLoader()
.getResource("yarn-site.xml");
if (url == null) {
throw new RuntimeException(
"Could not find 'yarn-site.xml' dummy file in classpath");
}
+ //write the document to a buffer (not directly to the file, as that
+ //can cause the file being written to get read -which will then fail.
+ ByteArrayOutputStream bytesOut = new ByteArrayOutputStream();
+ yarnClusterConfig.writeXml(bytesOut);
+ bytesOut.close();
+ //write the bytes to the file in the classpath
OutputStream os = new FileOutputStream(new File(url.getPath()));
- yarnCluster.getConfig().writeXml(os);
+ os.write(bytesOut.toByteArray());
os.close();
}
try {
@@ -74,8 +92,11 @@ public static void setup() throws InterruptedException, IOException {
@AfterClass
public static void tearDown() throws IOException {
if (yarnCluster != null) {
- yarnCluster.stop();
- yarnCluster = null;
+ try {
+ yarnCluster.stop();
+ } finally {
+ yarnCluster = null;
+ }
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/resources/log4j.properties b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/resources/log4j.properties
new file mode 100644
index 0000000000..315f1048ee
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/test/resources/log4j.properties
@@ -0,0 +1,37 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one
+# or more contributor license agreements. See the NOTICE file
+# distributed with this work for additional information
+# regarding copyright ownership. The ASF licenses this file
+# to you under the Apache License, Version 2.0 (the
+# "License"); you may not use this file except in compliance
+# with the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# Licensed under the Apache License, Version 2.0 (the "License");
+# you may not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+
+# log4j configuration used during build and unit tests
+
+log4j.rootLogger=INFO,stdout
+log4j.threshhold=ALL
+log4j.appender.stdout=org.apache.log4j.ConsoleAppender
+log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
+log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p [%t] %c{2} (%F:%M(%L)) - %m%n
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java
index f58e366247..ba0edabee6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientAsync.java
@@ -132,16 +132,16 @@ public AMRMClientAsync(AMRMClient client, int intervalMs,
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
client.init(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
handlerThread.start();
client.start();
- super.start();
+ super.serviceStart();
}
/**
@@ -150,7 +150,7 @@ public void start() {
* deadlock, and thus should be avoided.
*/
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (Thread.currentThread() == handlerThread) {
throw new YarnRuntimeException("Cannot call stop from callback handler thread!");
}
@@ -167,7 +167,7 @@ public void stop() {
} catch (InterruptedException ex) {
LOG.error("Error joining with hander thread", ex);
}
- super.stop();
+ super.serviceStop();
}
public void setHeartbeatInterval(int interval) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java
index 9562135cec..aef0e5b926 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/AMRMClientImpl.java
@@ -151,12 +151,12 @@ public AMRMClientImpl(ApplicationAttemptId appAttemptId) {
}
@Override
- public synchronized void init(Configuration conf) {
- super.init(conf);
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
final YarnConfiguration conf = new YarnConfiguration(getConfig());
final YarnRPC rpc = YarnRPC.create(conf);
final InetSocketAddress rmAddress = conf.getSocketAddr(
@@ -180,15 +180,15 @@ public AMRMProtocol run() {
}
});
LOG.debug("Connecting to ResourceManager at " + rmAddress);
- super.start();
+ super.serviceStart();
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
if (this.rmClient != null) {
RPC.stopProxy(this.rmClient);
}
- super.stop();
+ super.serviceStop();
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java
index 372ce22e15..653a2dccb0 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientAsync.java
@@ -166,18 +166,18 @@ protected NMClientAsync(String name, NMClient client,
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
this.maxThreadPoolSize = conf.getInt(
YarnConfiguration.NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE,
YarnConfiguration.DEFAULT_NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE);
LOG.info("Upper bound of the thread pool size is " + maxThreadPoolSize);
client.init(conf);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
client.start();
ThreadFactory tf = new ThreadFactoryBuilder().setNameFormat(
@@ -243,31 +243,39 @@ public void run() {
eventDispatcherThread.setDaemon(false);
eventDispatcherThread.start();
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (stopped.getAndSet(true)) {
// return if already stopped
return;
}
- eventDispatcherThread.interrupt();
- try {
- eventDispatcherThread.join();
- } catch (InterruptedException e) {
- LOG.error("The thread of " + eventDispatcherThread.getName() +
- " didn't finish normally.", e);
+ if (eventDispatcherThread != null) {
+ eventDispatcherThread.interrupt();
+ try {
+ eventDispatcherThread.join();
+ } catch (InterruptedException e) {
+ LOG.error("The thread of " + eventDispatcherThread.getName() +
+ " didn't finish normally.", e);
+ }
}
- threadPool.shutdownNow();
- // If NMClientImpl doesn't stop running containers, the states doesn't
- // need to be cleared.
- if (!(client instanceof NMClientImpl) ||
- ((NMClientImpl) client).cleanupRunningContainers.get()) {
- containers.clear();
+ if (threadPool != null) {
+ threadPool.shutdownNow();
}
- client.stop();
- super.stop();
+ if (client != null) {
+ // If NMClientImpl doesn't stop running containers, the states doesn't
+ // need to be cleared.
+ if (!(client instanceof NMClientImpl) ||
+ ((NMClientImpl) client).cleanupRunningContainers.get()) {
+ if (containers != null) {
+ containers.clear();
+ }
+ }
+ client.stop();
+ }
+ super.serviceStop();
}
public void startContainer(
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java
index 5c4b810aaa..b1be764ccd 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/NMClientImpl.java
@@ -86,7 +86,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
new ConcurrentHashMap();
//enabled by default
- protected AtomicBoolean cleanupRunningContainers = new AtomicBoolean(true);
+ protected final AtomicBoolean cleanupRunningContainers = new AtomicBoolean(true);
public NMClientImpl() {
super(NMClientImpl.class.getName());
@@ -97,13 +97,13 @@ public NMClientImpl(String name) {
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
// Usually, started-containers are stopped when this client stops. Unless
// the flag cleanupRunningContainers is set to false.
if (cleanupRunningContainers.get()) {
cleanupRunningContainers();
}
- super.stop();
+ super.serviceStop();
}
protected synchronized void cleanupRunningContainers() {
@@ -171,7 +171,7 @@ public NMCommunicator(ContainerId containerId, NodeId nodeId,
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
final YarnRPC rpc = YarnRPC.create(getConfig());
final InetSocketAddress containerAddress =
@@ -195,10 +195,11 @@ public ContainerManager run() {
});
LOG.debug("Connecting to ContainerManager at " + containerAddress);
+ super.serviceStart();
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
if (this.containerManager != null) {
RPC.stopProxy(this.containerManager);
@@ -209,6 +210,7 @@ public synchronized void stop() {
containerAddress);
}
}
+ super.serviceStop();
}
public synchronized Map startContainer(
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
index aea180c0b1..48be1a329c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/YarnClientImpl.java
@@ -79,7 +79,11 @@ public YarnClientImpl() {
}
public YarnClientImpl(InetSocketAddress rmAddress) {
- super(YarnClientImpl.class.getName());
+ this(YarnClientImpl.class.getName(), rmAddress);
+ }
+
+ public YarnClientImpl(String name, InetSocketAddress rmAddress) {
+ super(name);
this.rmAddress = rmAddress;
}
@@ -89,18 +93,18 @@ private static InetSocketAddress getRmAddress(Configuration conf) {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
if (this.rmAddress == null) {
this.rmAddress = getRmAddress(conf);
}
statePollIntervalMillis = conf.getLong(
YarnConfiguration.YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS,
YarnConfiguration.DEFAULT_YARN_CLIENT_APP_SUBMISSION_POLL_INTERVAL_MS);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
YarnRPC rpc = YarnRPC.create(getConfig());
this.rmClient = (ClientRMProtocol) rpc.getProxy(
@@ -108,15 +112,15 @@ public synchronized void start() {
if (LOG.isDebugEnabled()) {
LOG.debug("Connecting to ResourceManager at " + rmAddress);
}
- super.start();
+ super.serviceStart();
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
if (this.rmClient != null) {
RPC.stopProxy(this.rmClient);
}
- super.stop();
+ super.serviceStop();
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
index 6f46ded572..86a3eb62e1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClient.java
@@ -153,24 +153,37 @@ public void setup() throws YarnException, IOException {
@After
public void tearDown() {
rmClient.stop();
+ yarnClient.stop();
+ yarnCluster.stop();
+ }
+ private void stopNmClient(boolean stopContainers) {
+ assertNotNull("Null nmClient", nmClient);
// leave one unclosed
assertEquals(1, nmClient.startedContainers.size());
// default true
assertTrue(nmClient.cleanupRunningContainers.get());
- // don't stop the running containers
- nmClient.cleanupRunningContainersOnStop(false);
- assertFalse(nmClient.cleanupRunningContainers.get());
+ nmClient.cleanupRunningContainersOnStop(stopContainers);
+ assertEquals(stopContainers, nmClient.cleanupRunningContainers.get());
nmClient.stop();
- assertTrue(nmClient.startedContainers.size() > 0);
- // stop the running containers
- nmClient.cleanupRunningContainersOnStop(true);
- assertTrue(nmClient.cleanupRunningContainers.get());
- nmClient.stop();
- assertEquals(0, nmClient.startedContainers.size());
+ }
- yarnClient.stop();
- yarnCluster.stop();
+ @Test (timeout = 60000)
+ public void testNMClientNoCleanupOnStop()
+ throws YarnException, IOException {
+
+ rmClient.registerApplicationMaster("Host", 10000, "");
+
+ testContainerManagement(nmClient, allocateContainers(rmClient, 5));
+
+ rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
+ null, null);
+ // don't stop the running containers
+ stopNmClient(false);
+ assertFalse(nmClient.startedContainers. isEmpty());
+ //now cleanup
+ nmClient.cleanupRunningContainers();
+ assertEquals(0, nmClient.startedContainers.size());
}
@Test (timeout = 60000)
@@ -183,6 +196,11 @@ public void testNMClient()
rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
null, null);
+ // stop the running containers on close
+ assertFalse(nmClient.startedContainers.isEmpty());
+ nmClient.cleanupRunningContainersOnStop(true);
+ assertTrue(nmClient.cleanupRunningContainers.get());
+ nmClient.stop();
}
private Set allocateContainers(
@@ -250,9 +268,12 @@ private void testContainerManagement(NMClientImpl nmClient,
container.getContainerToken());
fail("Exception is expected");
} catch (YarnException e) {
- assertTrue("The thrown exception is not expected",
- e.getMessage().contains(
- "is either not started yet or already stopped"));
+ if (!e.getMessage()
+ .contains("is either not started yet or already stopped")) {
+ throw (AssertionError)
+ (new AssertionError("Exception is not expected: " + e).initCause(
+ e));
+ }
}
Credentials ts = new Credentials();
@@ -266,7 +287,8 @@ private void testContainerManagement(NMClientImpl nmClient,
try {
nmClient.startContainer(container, clc);
} catch (YarnException e) {
- fail("Exception is not expected");
+ throw (AssertionError)
+ (new AssertionError("Exception is not expected: " + e).initCause(e));
}
// leave one container unclosed
@@ -279,7 +301,9 @@ private void testContainerManagement(NMClientImpl nmClient,
nmClient.stopContainer(container.getId(), container.getNodeId(),
container.getContainerToken());
} catch (YarnException e) {
- fail("Exception is not expected");
+ throw (AssertionError)
+ (new AssertionError("Exception is not expected: " + e)
+ .initCause(e));
}
// getContainerStatus can be called after stopContainer
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
index 6fff279727..939072da8c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/TestNMClientAsync.java
@@ -52,6 +52,8 @@
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.RPCUtil;
+import org.apache.hadoop.yarn.service.ServiceOperations;
+import org.junit.After;
import org.junit.Test;
@@ -64,6 +66,11 @@ public class TestNMClientAsync {
private NodeId nodeId;
private Token containerToken;
+ @After
+ public void teardown() {
+ ServiceOperations.stop(asyncClient);
+ }
+
@Test (timeout = 30000)
public void testNMClientAsync() throws Exception {
Configuration conf = new Configuration();
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
index a8a9be4524..1f29e0369d 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/event/AsyncDispatcher.java
@@ -82,24 +82,24 @@ public void run() {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
this.exitOnDispatchException =
conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
//start all the components
- super.start();
+ super.serviceStart();
eventHandlingThread = new Thread(createThread());
eventHandlingThread.setName("AsyncDispatcher event handler");
eventHandlingThread.start();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
stopped = true;
if (eventHandlingThread != null) {
eventHandlingThread.interrupt();
@@ -111,7 +111,7 @@ public void stop() {
}
// stop all the components
- super.stop();
+ super.serviceStop();
}
@SuppressWarnings("unchecked")
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
index c8603ab7c1..c9dc580e19 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/logaggregation/AggregatedLogDeletionService.java
@@ -125,8 +125,9 @@ private static void logIOException(String comment, IOException e) {
public AggregatedLogDeletionService() {
super(AggregatedLogDeletionService.class.getName());
}
-
- public void start() {
+
+ @Override
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
if (!conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED)) {
@@ -150,14 +151,14 @@ public void start() {
TimerTask task = new LogDeletionTask(conf, retentionSecs);
timer = new Timer();
timer.scheduleAtFixedRate(task, 0, checkIntervalMsecs);
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if(timer != null) {
timer.cancel();
}
- super.stop();
+ super.serviceStop();
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/AbstractService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/AbstractService.java
index eeea1e1705..fc35f1c7c2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/AbstractService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/AbstractService.java
@@ -18,26 +18,33 @@
package org.apache.hadoop.yarn.service;
+import java.io.IOException;
import java.util.ArrayList;
+import java.util.HashMap;
import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+/**
+ * This is the base implementation class for YARN services.
+ */
public abstract class AbstractService implements Service {
private static final Log LOG = LogFactory.getLog(AbstractService.class);
- /**
- * Service state: initially {@link STATE#NOTINITED}.
- */
- private STATE state = STATE.NOTINITED;
-
/**
* Service name.
*/
private final String name;
+
+ /** service state */
+ private final ServiceStateModel stateModel;
+
/**
* Service start time. Will be zero until the service is started.
*/
@@ -46,83 +53,341 @@ public abstract class AbstractService implements Service {
/**
* The configuration. Will be null until the service is initialized.
*/
- private Configuration config;
+ private volatile Configuration config;
/**
* List of state change listeners; it is final to ensure
* that it will never be null.
*/
- private List listeners =
- new ArrayList();
+ private final ServiceOperations.ServiceListeners listeners
+ = new ServiceOperations.ServiceListeners();
+ /**
+ * Static listeners to all events across all services
+ */
+ private static ServiceOperations.ServiceListeners globalListeners
+ = new ServiceOperations.ServiceListeners();
+ /**
+ * The cause of any failure -will be null.
+ * if a service did not stop due to a failure.
+ */
+ private Exception failureCause;
+
+ /**
+ * the state in which the service was when it failed.
+ * Only valid when the service is stopped due to a failure
+ */
+ private STATE failureState = null;
+
+ /**
+ * object used to co-ordinate {@link #waitForServiceToStop(long)}
+ * across threads.
+ */
+ private final AtomicBoolean terminationNotification =
+ new AtomicBoolean(false);
+
+ /**
+ * History of lifecycle transitions
+ */
+ private final List lifecycleHistory
+ = new ArrayList(5);
+
+ /**
+ * Map of blocking dependencies
+ */
+ private final Map blockerMap = new HashMap();
+
+ private final Object stateChangeLock = new Object();
+
/**
* Construct the service.
* @param name service name
*/
public AbstractService(String name) {
this.name = name;
+ stateModel = new ServiceStateModel(name);
}
@Override
- public synchronized STATE getServiceState() {
- return state;
+ public final STATE getServiceState() {
+ return stateModel.getState();
+ }
+
+ @Override
+ public final synchronized Throwable getFailureCause() {
+ return failureCause;
+ }
+
+ @Override
+ public synchronized STATE getFailureState() {
+ return failureState;
}
/**
- * {@inheritDoc}
- * @throws IllegalStateException if the current service state does not permit
- * this action
+ * Set the configuration for this service.
+ * This method is called during {@link #init(Configuration)}
+ * and should only be needed if for some reason a service implementation
+ * needs to override that initial setting -for example replacing
+ * it with a new subclass of {@link Configuration}
+ * @param conf new configuration.
*/
- @Override
- public synchronized void init(Configuration conf) {
- ensureCurrentState(STATE.NOTINITED);
+ protected void setConfig(Configuration conf) {
this.config = conf;
- changeState(STATE.INITED);
- LOG.info("Service:" + getName() + " is inited.");
}
/**
* {@inheritDoc}
- * @throws IllegalStateException if the current service state does not permit
- * this action
+ * This invokes {@link #serviceInit}
+ * @param conf the configuration of the service. This must not be null
+ * @throws ServiceStateException if the configuration was null,
+ * the state change not permitted, or something else went wrong
*/
@Override
- public synchronized void start() {
- startTime = System.currentTimeMillis();
- ensureCurrentState(STATE.INITED);
- changeState(STATE.STARTED);
- LOG.info("Service:" + getName() + " is started.");
- }
-
- /**
- * {@inheritDoc}
- * @throws IllegalStateException if the current service state does not permit
- * this action
- */
- @Override
- public synchronized void stop() {
- if (state == STATE.STOPPED ||
- state == STATE.INITED ||
- state == STATE.NOTINITED) {
- // already stopped, or else it was never
- // started (eg another service failing canceled startup)
+ public void init(Configuration conf) {
+ if (conf == null) {
+ throw new ServiceStateException("Cannot initialize service "
+ + getName() + ": null configuration");
+ }
+ if (isInState(STATE.INITED)) {
return;
}
- ensureCurrentState(STATE.STARTED);
- changeState(STATE.STOPPED);
- LOG.info("Service:" + getName() + " is stopped.");
+ synchronized (stateChangeLock) {
+ if (enterState(STATE.INITED) != STATE.INITED) {
+ setConfig(conf);
+ try {
+ serviceInit(config);
+ if (isInState(STATE.INITED)) {
+ //if the service ended up here during init,
+ //notify the listeners
+ notifyListeners();
+ }
+ } catch (Exception e) {
+ noteFailure(e);
+ ServiceOperations.stopQuietly(LOG, this);
+ throw ServiceStateException.convert(e);
+ }
+ }
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ * @throws ServiceStateException if the current service state does not permit
+ * this action
+ */
+ @Override
+ public void start() {
+ if (isInState(STATE.STARTED)) {
+ return;
+ }
+ //enter the started state
+ synchronized (stateChangeLock) {
+ if (stateModel.enterState(STATE.STARTED) != STATE.STARTED) {
+ try {
+ startTime = System.currentTimeMillis();
+ serviceStart();
+ if (isInState(STATE.STARTED)) {
+ //if the service started (and isn't now in a later state), notify
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Service " + getName() + " is started");
+ }
+ notifyListeners();
+ }
+ } catch (Exception e) {
+ noteFailure(e);
+ ServiceOperations.stopQuietly(LOG, this);
+ throw ServiceStateException.convert(e);
+ }
+ }
+ }
+ }
+
+ /**
+ * {@inheritDoc}
+ */
+ @Override
+ public void stop() {
+ if (isInState(STATE.STOPPED)) {
+ return;
+ }
+ synchronized (stateChangeLock) {
+ if (enterState(STATE.STOPPED) != STATE.STOPPED) {
+ try {
+ serviceStop();
+ } catch (Exception e) {
+ //stop-time exceptions are logged if they are the first one,
+ noteFailure(e);
+ throw ServiceStateException.convert(e);
+ } finally {
+ //report that the service has terminated
+ terminationNotification.set(true);
+ synchronized (terminationNotification) {
+ terminationNotification.notifyAll();
+ }
+ //notify anything listening for events
+ notifyListeners();
+ }
+ } else {
+ //already stopped: note it
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Ignoring re-entrant call to stop()");
+ }
+ }
+ }
+ }
+
+ /**
+ * Relay to {@link #stop()}
+ * @throws IOException
+ */
+ @Override
+ public final void close() throws IOException {
+ stop();
+ }
+
+ /**
+ * Failure handling: record the exception
+ * that triggered it -if there was not one already.
+ * Services are free to call this themselves.
+ * @param exception the exception
+ */
+ protected final void noteFailure(Exception exception) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("noteFailure " + exception, null);
+ }
+ if (exception == null) {
+ //make sure failure logic doesn't itself cause problems
+ return;
+ }
+ //record the failure details, and log it
+ synchronized (this) {
+ if (failureCause == null) {
+ failureCause = exception;
+ failureState = getServiceState();
+ LOG.info("Service " + getName()
+ + " failed in state " + failureState
+ + "; cause: " + exception,
+ exception);
+ }
+ }
}
@Override
- public synchronized void register(ServiceStateChangeListener l) {
+ public final boolean waitForServiceToStop(long timeout) {
+ boolean completed = terminationNotification.get();
+ while (!completed) {
+ try {
+ synchronized(terminationNotification) {
+ terminationNotification.wait(timeout);
+ }
+ // here there has been a timeout, the object has terminated,
+ // or there has been a spurious wakeup (which we ignore)
+ completed = true;
+ } catch (InterruptedException e) {
+ // interrupted; have another look at the flag
+ completed = terminationNotification.get();
+ }
+ }
+ return terminationNotification.get();
+ }
+
+ /* ===================================================================== */
+ /* Override Points */
+ /* ===================================================================== */
+
+ /**
+ * All initialization code needed by a service.
+ *
+ * This method will only ever be called once during the lifecycle of
+ * a specific service instance.
+ *
+ * Implementations do not need to be synchronized as the logic
+ * in {@link #init(Configuration)} prevents re-entrancy.
+ *
+ * The base implementation checks to see if the subclass has created
+ * a new configuration instance, and if so, updates the base class value
+ * @param conf configuration
+ * @throws Exception on a failure -these will be caught,
+ * possibly wrapped, and wil; trigger a service stop
+ */
+ protected void serviceInit(Configuration conf) throws Exception {
+ if (conf != config) {
+ LOG.debug("Config has been overridden during init");
+ setConfig(conf);
+ }
+ }
+
+ /**
+ * Actions called during the INITED to STARTED transition.
+ *
+ * This method will only ever be called once during the lifecycle of
+ * a specific service instance.
+ *
+ * Implementations do not need to be synchronized as the logic
+ * in {@link #start()} prevents re-entrancy.
+ *
+ * @throws Exception if needed -these will be caught,
+ * wrapped, and trigger a service stop
+ */
+ protected void serviceStart() throws Exception {
+
+ }
+
+ /**
+ * Actions called during the transition to the STOPPED state.
+ *
+ * This method will only ever be called once during the lifecycle of
+ * a specific service instance.
+ *
+ * Implementations do not need to be synchronized as the logic
+ * in {@link #stop()} prevents re-entrancy.
+ *
+ * Implementations MUST write this to be robust against failures, including
+ * checks for null references -and for the first failure to not stop other
+ * attempts to shut down parts of the service.
+ *
+ * @throws Exception if needed -these will be caught and logged.
+ */
+ protected void serviceStop() throws Exception {
+
+ }
+
+ @Override
+ public void register(ServiceStateChangeListener l) {
listeners.add(l);
}
@Override
- public synchronized void unregister(ServiceStateChangeListener l) {
+ public void unregister(ServiceStateChangeListener l) {
listeners.remove(l);
}
+ /**
+ * Register a global listener, which receives notifications
+ * from the state change events of all services in the JVM
+ * @param l listener
+ */
+ public static void registerGlobalListener(ServiceStateChangeListener l) {
+ globalListeners.add(l);
+ }
+
+ /**
+ * unregister a global listener.
+ * @param l listener to unregister
+ * @return true if the listener was found (and then deleted)
+ */
+ public static boolean unregisterGlobalListener(ServiceStateChangeListener l) {
+ return globalListeners.remove(l);
+ }
+
+ /**
+ * Package-scoped method for testing -resets the global listener list
+ */
+ @VisibleForTesting
+ static void resetGlobalListeners() {
+ globalListeners.reset();
+ }
+
@Override
public String getName() {
return name;
@@ -139,28 +404,92 @@ public long getStartTime() {
}
/**
- * Verify that that a service is in a given state.
- * @param currentState the desired state
- * @throws IllegalStateException if the service state is different from
- * the desired state
+ * Notify local and global listeners of state changes.
+ * Exceptions raised by listeners are NOT passed up.
*/
- private void ensureCurrentState(STATE currentState) {
- ServiceOperations.ensureCurrentState(state, currentState);
+ private void notifyListeners() {
+ try {
+ listeners.notifyListeners(this);
+ globalListeners.notifyListeners(this);
+ } catch (Throwable e) {
+ LOG.warn("Exception while notifying listeners of " + this + ": " + e,
+ e);
+ }
}
/**
- * Change to a new state and notify all listeners.
- * This is a private method that is only invoked from synchronized methods,
- * which avoid having to clone the listener list. It does imply that
- * the state change listener methods should be short lived, as they
- * will delay the state transition.
- * @param newState new service state
+ * Add a state change event to the lifecycle history
*/
- private void changeState(STATE newState) {
- state = newState;
- //notify listeners
- for (ServiceStateChangeListener l : listeners) {
- l.stateChanged(this);
+ private void recordLifecycleEvent() {
+ LifecycleEvent event = new LifecycleEvent();
+ event.time = System.currentTimeMillis();
+ event.state = getServiceState();
+ lifecycleHistory.add(event);
+ }
+
+ @Override
+ public synchronized List getLifecycleHistory() {
+ return new ArrayList(lifecycleHistory);
+ }
+
+ /**
+ * Enter a state; record this via {@link #recordLifecycleEvent}
+ * and log at the info level.
+ * @param newState the proposed new state
+ * @return the original state
+ * it wasn't already in that state, and the state model permits state re-entrancy.
+ */
+ private STATE enterState(STATE newState) {
+ assert stateModel != null : "null state in " + name + " " + this.getClass();
+ STATE oldState = stateModel.enterState(newState);
+ if (oldState != newState) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(
+ "Service: " + getName() + " entered state " + getServiceState());
+ }
+ recordLifecycleEvent();
+ }
+ return oldState;
+ }
+
+ @Override
+ public final boolean isInState(Service.STATE expected) {
+ return stateModel.isInState(expected);
+ }
+
+ @Override
+ public String toString() {
+ return "Service " + name + " in state " + stateModel;
+ }
+
+ /**
+ * Put a blocker to the blocker map -replacing any
+ * with the same name.
+ * @param name blocker name
+ * @param details any specifics on the block. This must be non-null.
+ */
+ protected void putBlocker(String name, String details) {
+ synchronized (blockerMap) {
+ blockerMap.put(name, details);
+ }
+ }
+
+ /**
+ * Remove a blocker from the blocker map -
+ * this is a no-op if the blocker is not present
+ * @param name the name of the blocker
+ */
+ public void removeBlocker(String name) {
+ synchronized (blockerMap) {
+ blockerMap.remove(name);
+ }
+ }
+
+ @Override
+ public Map getBlockers() {
+ synchronized (blockerMap) {
+ Map map = new HashMap(blockerMap);
+ return map;
}
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java
index 26a091d586..0e6603b6f5 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/CompositeService.java
@@ -19,14 +19,12 @@
package org.apache.hadoop.yarn.service;
import java.util.ArrayList;
-import java.util.Collection;
import java.util.Collections;
import java.util.List;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.YarnRuntimeException;
/**
* Composition of services.
@@ -35,72 +33,115 @@ public class CompositeService extends AbstractService {
private static final Log LOG = LogFactory.getLog(CompositeService.class);
- private List serviceList = new ArrayList();
+ /**
+ * Policy on shutdown: attempt to close everything (purest) or
+ * only try to close started services (which assumes
+ * that the service implementations may not handle the stop() operation
+ * except when started.
+ * Irrespective of this policy, if a child service fails during
+ * its init() or start() operations, it will have stop() called on it.
+ */
+ protected static final boolean STOP_ONLY_STARTED_SERVICES = false;
+
+ private final List serviceList = new ArrayList();
public CompositeService(String name) {
super(name);
}
- public Collection getServices() {
- return Collections.unmodifiableList(serviceList);
+ /**
+ * Get an unmodifiable list of services
+ * @return a list of child services at the time of invocation -
+ * added services will not be picked up.
+ */
+ public List getServices() {
+ synchronized (serviceList) {
+ return Collections.unmodifiableList(serviceList);
+ }
}
- protected synchronized void addService(Service service) {
- serviceList.add(service);
+ protected void addService(Service service) {
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Adding service " + service.getName());
+ }
+ synchronized (serviceList) {
+ serviceList.add(service);
+ }
}
protected synchronized boolean removeService(Service service) {
- return serviceList.remove(service);
+ synchronized (serviceList) {
+ return serviceList.add(service);
+ }
}
- public synchronized void init(Configuration conf) {
- for (Service service : serviceList) {
+ protected void serviceInit(Configuration conf) throws Exception {
+ List services = getServices();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(getName() + ": initing services, size=" + services.size());
+ }
+ for (Service service : services) {
service.init(conf);
}
- super.init(conf);
+ super.serviceInit(conf);
}
- public synchronized void start() {
- int i = 0;
- try {
- for (int n = serviceList.size(); i < n; i++) {
- Service service = serviceList.get(i);
- service.start();
+ protected void serviceStart() throws Exception {
+ List services = getServices();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(getName() + ": starting services, size=" + services.size());
+ }
+ for (Service service : services) {
+ // start the service. If this fails that service
+ // will be stopped and an exception raised
+ service.start();
+ }
+ super.serviceStart();
+ }
+
+ protected void serviceStop() throws Exception {
+ //stop all services that were started
+ int numOfServicesToStop = serviceList.size();
+ if (LOG.isDebugEnabled()) {
+ LOG.debug(getName() + ": stopping services, size=" + numOfServicesToStop);
+ }
+ stop(numOfServicesToStop, STOP_ONLY_STARTED_SERVICES);
+ super.serviceStop();
+ }
+
+ /**
+ * Stop the services in reverse order
+ *
+ * @param numOfServicesStarted index from where the stop should work
+ * @param stopOnlyStartedServices flag to say "only start services that are
+ * started, not those that are NOTINITED or INITED.
+ * @throws RuntimeException the first exception raised during the
+ * stop process -after all services are stopped
+ */
+ private synchronized void stop(int numOfServicesStarted,
+ boolean stopOnlyStartedServices) {
+ // stop in reverse order of start
+ Exception firstException = null;
+ List services = getServices();
+ for (int i = numOfServicesStarted - 1; i >= 0; i--) {
+ Service service = services.get(i);
+ if (LOG.isDebugEnabled()) {
+ LOG.debug("Stopping service #" + i + ": " + service);
}
- super.start();
- } catch (Throwable e) {
- LOG.error("Error starting services " + getName(), e);
- // Note that the state of the failed service is still INITED and not
- // STARTED. Even though the last service is not started completely, still
- // call stop() on all services including failed service to make sure cleanup
- // happens.
- stop(i);
- throw new YarnRuntimeException("Failed to Start " + getName(), e);
- }
-
- }
-
- public synchronized void stop() {
- if (this.getServiceState() == STATE.STOPPED) {
- // The base composite-service is already stopped, don't do anything again.
- return;
- }
- if (serviceList.size() > 0) {
- stop(serviceList.size() - 1);
- }
- super.stop();
- }
-
- private synchronized void stop(int numOfServicesStarted) {
- // stop in reserve order of start
- for (int i = numOfServicesStarted; i >= 0; i--) {
- Service service = serviceList.get(i);
- try {
- service.stop();
- } catch (Throwable t) {
- LOG.info("Error stopping " + service.getName(), t);
+ STATE state = service.getServiceState();
+ //depending on the stop police
+ if (state == STATE.STARTED
+ || (!stopOnlyStartedServices && state == STATE.INITED)) {
+ Exception ex = ServiceOperations.stopQuietly(LOG, service);
+ if (ex != null && firstException == null) {
+ firstException = ex;
+ }
}
}
+ //after stopping all services, rethrow the first exception raised
+ if (firstException != null) {
+ throw ServiceStateException.convert(firstException);
+ }
}
/**
@@ -117,13 +158,8 @@ public CompositeServiceShutdownHook(CompositeService compositeService) {
@Override
public void run() {
- try {
- // Stop the Composite Service
- compositeService.stop();
- } catch (Throwable t) {
- LOG.info("Error stopping " + compositeService.getName(), t);
- }
+ ServiceOperations.stopQuietly(compositeService);
}
}
-
+
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/FilterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/FilterService.java
index 314d6647ec..07aafaf6ef 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/FilterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/FilterService.java
@@ -20,6 +20,10 @@
import org.apache.hadoop.conf.Configuration;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
public class FilterService implements Service {
private final Service service;
@@ -44,6 +48,11 @@ public void stop() {
service.stop();
}
+ @Override
+ public void close() throws IOException {
+ service.close();
+ }
+
@Override
public void register(ServiceStateChangeListener listener) {
service.register(listener);
@@ -73,4 +82,34 @@ public STATE getServiceState() {
public long getStartTime() {
return startTime;
}
+
+ @Override
+ public boolean isInState(STATE state) {
+ return service.isInState(state);
+ }
+
+ @Override
+ public Throwable getFailureCause() {
+ return service.getFailureCause();
+ }
+
+ @Override
+ public STATE getFailureState() {
+ return service.getFailureState();
+ }
+
+ @Override
+ public boolean waitForServiceToStop(long timeout) {
+ return service.waitForServiceToStop(timeout);
+ }
+
+ @Override
+ public List getLifecycleHistory() {
+ return service.getLifecycleHistory();
+ }
+
+ @Override
+ public Map getBlockers() {
+ return service.getBlockers();
+ }
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/LifecycleEvent.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/LifecycleEvent.java
new file mode 100644
index 0000000000..4acd4a1b20
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/LifecycleEvent.java
@@ -0,0 +1,35 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.yarn.service;
+
+import java.io.Serializable;
+
+/**
+ * A serializable lifecycle event: the time a state
+ * transition occurred, and what state was entered.
+ */
+public class LifecycleEvent implements Serializable {
+ /**
+ * Local time in milliseconds when the event occurred
+ */
+ public long time;
+ /**
+ * new state
+ */
+ public Service.STATE state;
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/LoggingStateChangeListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/LoggingStateChangeListener.java
new file mode 100644
index 0000000000..b0c3af100e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/LoggingStateChangeListener.java
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+
+/**
+ * This is a state change listener that logs events at INFO level
+ */
+public class LoggingStateChangeListener implements ServiceStateChangeListener {
+
+ private static final Log LOG = LogFactory.getLog(LoggingStateChangeListener.class);
+
+ private final Log log;
+
+ /**
+ * Log events to the given log
+ * @param log destination for events
+ */
+ public LoggingStateChangeListener(Log log) {
+ //force an NPE if a null log came in
+ log.isDebugEnabled();
+ this.log = log;
+ }
+
+ /**
+ * Log events to the static log for this class
+ */
+ public LoggingStateChangeListener() {
+ this(LOG);
+ }
+
+ /**
+ * Callback for a state change event: log it
+ * @param service the service that has changed.
+ */
+ @Override
+ public void stateChanged(Service service) {
+ log.info("Entry to state " + service.getServiceState()
+ + " for " + service.getName());
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/Service.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/Service.java
index 8b8b1837dd..1749bf9bfa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/Service.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/Service.java
@@ -20,34 +20,77 @@
import org.apache.hadoop.conf.Configuration;
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
/**
* Service LifeCycle.
*/
-public interface Service {
+public interface Service extends Closeable {
/**
* Service states
*/
public enum STATE {
/** Constructed but not initialized */
- NOTINITED,
+ NOTINITED(0, "NOTINITED"),
/** Initialized but not started or stopped */
- INITED,
+ INITED(1, "INITED"),
/** started and not stopped */
- STARTED,
+ STARTED(2, "STARTED"),
/** stopped. No further state transitions are permitted */
- STOPPED
+ STOPPED(3, "STOPPED");
+
+ /**
+ * An integer value for use in array lookup and JMX interfaces.
+ * Although {@link Enum#ordinal()} could do this, explicitly
+ * identify the numbers gives more stability guarantees over time.
+ */
+ private final int value;
+
+ /**
+ * A name of the state that can be used in messages
+ */
+ private final String statename;
+
+ private STATE(int value, String name) {
+ this.value = value;
+ this.statename = name;
+ }
+
+ /**
+ * Get the integer value of a state
+ * @return the numeric value of the state
+ */
+ public int getValue() {
+ return value;
+ }
+
+ /**
+ * Get the name of a state
+ * @return the state's name
+ */
+ @Override
+ public String toString() {
+ return statename;
+ }
}
/**
* Initialize the service.
*
- * The transition must be from {@link STATE#NOTINITED} to {@link STATE#INITED}
- * unless the operation failed and an exception was raised.
+ * The transition MUST be from {@link STATE#NOTINITED} to {@link STATE#INITED}
+ * unless the operation failed and an exception was raised, in which case
+ * {@link #stop()} MUST be invoked and the service enter the state
+ * {@link STATE#STOPPED}.
* @param config the configuration of the service
+ * @throws RuntimeException on any failure during the operation
+
*/
void init(Configuration config);
@@ -55,20 +98,36 @@ public enum STATE {
/**
* Start the service.
*
- * The transition should be from {@link STATE#INITED} to {@link STATE#STARTED}
- * unless the operation failed and an exception was raised.
+ * The transition MUST be from {@link STATE#INITED} to {@link STATE#STARTED}
+ * unless the operation failed and an exception was raised, in which case
+ * {@link #stop()} MUST be invoked and the service enter the state
+ * {@link STATE#STOPPED}.
+ * @throws RuntimeException on any failure during the operation
*/
void start();
/**
- * Stop the service.
+ * Stop the service. This MUST be a no-op if the service is already
+ * in the {@link STATE#STOPPED} state. It SHOULD be a best-effort attempt
+ * to stop all parts of the service.
*
- * This operation must be designed to complete regardless of the initial state
- * of the service, including the state of all its internal fields.
+ * The implementation must be designed to complete regardless of the service
+ * state, including the initialized/uninitialized state of all its internal
+ * fields.
+ * @throws RuntimeException on any failure during the stop operation
*/
void stop();
+ /**
+ * A version of stop() that is designed to be usable in Java7 closure
+ * clauses.
+ * Implementation classes MUST relay this directly to {@link #stop()}
+ * @throws IOException never
+ * @throws RuntimeException on any failure during the stop operation
+ */
+ void close() throws IOException;
+
/**
* Register an instance of the service state change events.
* @param listener a new listener
@@ -108,4 +167,52 @@ public enum STATE {
* has not yet been started.
*/
long getStartTime();
+
+ /**
+ * Query to see if the service is in a specific state.
+ * In a multi-threaded system, the state may not hold for very long.
+ * @param state the expected state
+ * @return true if, at the time of invocation, the service was in that state.
+ */
+ boolean isInState(STATE state);
+
+ /**
+ * Get the first exception raised during the service failure. If null,
+ * no exception was logged
+ * @return the failure logged during a transition to the stopped state
+ */
+ Throwable getFailureCause();
+
+ /**
+ * Get the state in which the failure in {@link #getFailureCause()} occurred.
+ * @return the state or null if there was no failure
+ */
+ STATE getFailureState();
+
+ /**
+ * Block waiting for the service to stop; uses the termination notification
+ * object to do so.
+ *
+ * This method will only return after all the service stop actions
+ * have been executed (to success or failure), or the timeout elapsed
+ * This method can be called before the service is inited or started; this is
+ * to eliminate any race condition with the service stopping before
+ * this event occurs.
+ * @param timeout timeout in milliseconds. A value of zero means "forever"
+ * @return true iff the service stopped in the time period
+ */
+ boolean waitForServiceToStop(long timeout);
+
+ /**
+ * Get a snapshot of the lifecycle history; it is a static list
+ * @return a possibly empty but never null list of lifecycle events.
+ */
+ public List getLifecycleHistory();
+
+ /**
+ * Get the blockers on a service -remote dependencies
+ * that are stopping the service from being live.
+ * @return a (snapshotted) map of blocker name->description values
+ */
+ public Map getBlockers();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceOperations.java
index 151caa9d16..24f9e85192 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceOperations.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceOperations.java
@@ -21,6 +21,11 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.util.ShutdownHookManager;
+
+import java.lang.ref.WeakReference;
+import java.util.ArrayList;
+import java.util.List;
/**
* This class contains a set of methods to work with services, especially
@@ -32,74 +37,6 @@ public final class ServiceOperations {
private ServiceOperations() {
}
- /**
- * Verify that that a service is in a given state.
- * @param state the actual state a service is in
- * @param expectedState the desired state
- * @throws IllegalStateException if the service state is different from
- * the desired state
- */
- public static void ensureCurrentState(Service.STATE state,
- Service.STATE expectedState) {
- if (state != expectedState) {
- throw new IllegalStateException("For this operation, the " +
- "current service state must be "
- + expectedState
- + " instead of " + state);
- }
- }
-
- /**
- * Initialize a service.
- *
- * The service state is checked before the operation begins.
- * This process is not thread safe.
- * @param service a service that must be in the state
- * {@link Service.STATE#NOTINITED}
- * @param configuration the configuration to initialize the service with
- * @throws RuntimeException on a state change failure
- * @throws IllegalStateException if the service is in the wrong state
- */
-
- public static void init(Service service, Configuration configuration) {
- Service.STATE state = service.getServiceState();
- ensureCurrentState(state, Service.STATE.NOTINITED);
- service.init(configuration);
- }
-
- /**
- * Start a service.
- *
- * The service state is checked before the operation begins.
- * This process is not thread safe.
- * @param service a service that must be in the state
- * {@link Service.STATE#INITED}
- * @throws RuntimeException on a state change failure
- * @throws IllegalStateException if the service is in the wrong state
- */
-
- public static void start(Service service) {
- Service.STATE state = service.getServiceState();
- ensureCurrentState(state, Service.STATE.INITED);
- service.start();
- }
-
- /**
- * Initialize then start a service.
- *
- * The service state is checked before the operation begins.
- * This process is not thread safe.
- * @param service a service that must be in the state
- * {@link Service.STATE#NOTINITED}
- * @param configuration the configuration to initialize the service with
- * @throws RuntimeException on a state change failure
- * @throws IllegalStateException if the service is in the wrong state
- */
- public static void deploy(Service service, Configuration configuration) {
- init(service, configuration);
- start(service);
- }
-
/**
* Stop a service.
* Do nothing if the service is null or not
@@ -111,10 +48,7 @@ public static void deploy(Service service, Configuration configuration) {
*/
public static void stop(Service service) {
if (service != null) {
- Service.STATE state = service.getServiceState();
- if (state == Service.STATE.STARTED) {
- service.stop();
- }
+ service.stop();
}
}
@@ -127,14 +61,93 @@ public static void stop(Service service) {
* @return any exception that was caught; null if none was.
*/
public static Exception stopQuietly(Service service) {
+ return stopQuietly(LOG, service);
+ }
+
+ /**
+ * Stop a service; if it is null do nothing. Exceptions are caught and
+ * logged at warn level. (but not Throwables). This operation is intended to
+ * be used in cleanup operations
+ *
+ * @param log the log to warn at
+ * @param service a service; may be null
+ * @return any exception that was caught; null if none was.
+ * @see ServiceOperations#stopQuietly(Service)
+ */
+ public static Exception stopQuietly(Log log, Service service) {
try {
stop(service);
} catch (Exception e) {
- LOG.warn("When stopping the service " + service.getName()
- + " : " + e,
+ log.warn("When stopping the service " + service.getName()
+ + " : " + e,
e);
return e;
}
return null;
}
+
+
+ /**
+ * Class to manage a list of {@link ServiceStateChangeListener} instances,
+ * including a notification loop that is robust against changes to the list
+ * during the notification process.
+ */
+ public static class ServiceListeners {
+ /**
+ * List of state change listeners; it is final to guarantee
+ * that it will never be null.
+ */
+ private final List listeners =
+ new ArrayList();
+
+ /**
+ * Thread-safe addition of a new listener to the end of a list.
+ * Attempts to re-register a listener that is already registered
+ * will be ignored.
+ * @param l listener
+ */
+ public synchronized void add(ServiceStateChangeListener l) {
+ if(!listeners.contains(l)) {
+ listeners.add(l);
+ }
+ }
+
+ /**
+ * Remove any registration of a listener from the listener list.
+ * @param l listener
+ * @return true if the listener was found (and then removed)
+ */
+ public synchronized boolean remove(ServiceStateChangeListener l) {
+ return listeners.remove(l);
+ }
+
+ /**
+ * Reset the listener list
+ */
+ public synchronized void reset() {
+ listeners.clear();
+ }
+
+ /**
+ * Change to a new state and notify all listeners.
+ * This method will block until all notifications have been issued.
+ * It caches the list of listeners before the notification begins,
+ * so additions or removal of listeners will not be visible.
+ * @param service the service that has changed state
+ */
+ public void notifyListeners(Service service) {
+ //take a very fast snapshot of the callback list
+ //very much like CopyOnWriteArrayList, only more minimal
+ ServiceStateChangeListener[] callbacks;
+ synchronized (this) {
+ callbacks = listeners.toArray(new ServiceStateChangeListener[listeners.size()]);
+ }
+ //iterate through the listeners outside the synchronized method,
+ //ensuring that listener registration/unregistration doesn't break anything
+ for (ServiceStateChangeListener l : callbacks) {
+ l.stateChanged(service);
+ }
+ }
+ }
+
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceStateException.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceStateException.java
new file mode 100644
index 0000000000..10608c6d89
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceStateException.java
@@ -0,0 +1,79 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service;
+
+import org.apache.hadoop.yarn.YarnRuntimeException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+
+/**
+ * Exception that is raised on state change operations.
+ */
+public class ServiceStateException extends YarnRuntimeException {
+
+ public ServiceStateException(String message) {
+ super(message);
+ }
+
+ public ServiceStateException(String message, Throwable cause) {
+ super(message, cause);
+ }
+
+ public ServiceStateException(Throwable cause) {
+ super(cause);
+ }
+
+ /**
+ * Convert any exception into a {@link RuntimeException}.
+ * If the caught exception already is of that type -including
+ * a {@link YarnException} it is typecast to a {@link RuntimeException}
+ * and returned.
+ *
+ * All other exception types are wrapped in a new instance of
+ * ServiceStateException
+ * @param fault exception or throwable
+ * @return a ServiceStateException to rethrow
+ */
+ public static RuntimeException convert(Throwable fault) {
+ if (fault instanceof RuntimeException) {
+ return (RuntimeException) fault;
+ } else {
+ return new ServiceStateException(fault);
+ }
+ }
+
+ /**
+ * Convert any exception into a {@link RuntimeException}.
+ * If the caught exception already is of that type -including
+ * a {@link YarnException} it is typecast to a {@link RuntimeException}
+ * and returned.
+ *
+ * All other exception types are wrapped in a new instance of
+ * ServiceStateException
+ * @param text text to use if a new exception is created
+ * @param fault exception or throwable
+ * @return a ServiceStateException to rethrow
+ */
+ public static RuntimeException convert(String text, Throwable fault) {
+ if (fault instanceof RuntimeException) {
+ return (RuntimeException) fault;
+ } else {
+ return new ServiceStateException(text, fault);
+ }
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceStateModel.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceStateModel.java
new file mode 100644
index 0000000000..a9dd138307
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/service/ServiceStateModel.java
@@ -0,0 +1,157 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service;
+
+/**
+ * Implements the service state model for YARN.
+ */
+public class ServiceStateModel {
+
+ /**
+ * Map of all valid state transitions
+ * [current] [proposed1, proposed2, ...]
+ */
+ private static final boolean[][] statemap =
+ {
+ // uninited inited started stopped
+ /* uninited */ {false, true, false, true},
+ /* inited */ {false, true, true, true},
+ /* started */ {false, false, true, true},
+ /* stopped */ {false, false, false, true},
+ };
+
+ /**
+ * The state of the service
+ */
+ private volatile Service.STATE state;
+
+ /**
+ * The name of the service: used in exceptions
+ */
+ private String name;
+
+ /**
+ * Create the service state model in the {@link Service.STATE#NOTINITED}
+ * state.
+ */
+ public ServiceStateModel(String name) {
+ this(name, Service.STATE.NOTINITED);
+ }
+
+ /**
+ * Create a service state model instance in the chosen state
+ * @param state the starting state
+ */
+ public ServiceStateModel(String name, Service.STATE state) {
+ this.state = state;
+ this.name = name;
+ }
+
+ /**
+ * Query the service state. This is a non-blocking operation.
+ * @return the state
+ */
+ public Service.STATE getState() {
+ return state;
+ }
+
+ /**
+ * Query that the state is in a specific state
+ * @param proposed proposed new state
+ * @return the state
+ */
+ public boolean isInState(Service.STATE proposed) {
+ return state.equals(proposed);
+ }
+
+ /**
+ * Verify that that a service is in a given state.
+ * @param expectedState the desired state
+ * @throws ServiceStateException if the service state is different from
+ * the desired state
+ */
+ public void ensureCurrentState(Service.STATE expectedState) {
+ if (state != expectedState) {
+ throw new ServiceStateException(name+ ": for this operation, the " +
+ "current service state must be "
+ + expectedState
+ + " instead of " + state);
+ }
+ }
+
+ /**
+ * Enter a state -thread safe.
+ *
+ * @param proposed proposed new state
+ * @return the original state
+ * @throws ServiceStateException if the transition is not permitted
+ */
+ public synchronized Service.STATE enterState(Service.STATE proposed) {
+ checkStateTransition(name, state, proposed);
+ Service.STATE oldState = state;
+ //atomic write of the new state
+ state = proposed;
+ return oldState;
+ }
+
+ /**
+ * Check that a state tansition is valid and
+ * throw an exception if not
+ * @param name name of the service (can be null)
+ * @param state current state
+ * @param proposed proposed new state
+ */
+ public static void checkStateTransition(String name,
+ Service.STATE state,
+ Service.STATE proposed) {
+ if (!isValidStateTransition(state, proposed)) {
+ throw new ServiceStateException(name + " cannot enter state "
+ + proposed + " from state " + state);
+ }
+ }
+
+ /**
+ * Is a state transition valid?
+ * There are no checks for current==proposed
+ * as that is considered a non-transition.
+ *
+ * using an array kills off all branch misprediction costs, at the expense
+ * of cache line misses.
+ *
+ * @param current current state
+ * @param proposed proposed new state
+ * @return true if the transition to a new state is valid
+ */
+ public static boolean isValidStateTransition(Service.STATE current,
+ Service.STATE proposed) {
+ boolean[] row = statemap[current.getValue()];
+ return row[proposed.getValue()];
+ }
+
+ /**
+ * return the state text as the toString() value
+ * @return the current state's description
+ */
+ @Override
+ public String toString() {
+ return (name.isEmpty() ? "" : ((name) + ": "))
+ + state.toString();
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java
index b46ad3efce..c6e4d22bb8 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/AbstractLivelinessMonitor.java
@@ -49,21 +49,21 @@ public AbstractLivelinessMonitor(String name, Clock clock) {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
assert !stopped : "starting when already stopped";
checkerThread = new Thread(new PingChecker());
checkerThread.setName("Ping Checker");
checkerThread.start();
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
stopped = true;
if (checkerThread != null) {
checkerThread.interrupt();
}
- super.stop();
+ super.serviceStop();
}
protected abstract void expire(O ob);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableService.java
index 5907f39d29..12c9649965 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableService.java
@@ -55,13 +55,7 @@ public BreakableService(boolean failOnInit,
}
private int convert(STATE state) {
- switch (state) {
- case NOTINITED: return 0;
- case INITED: return 1;
- case STARTED: return 2;
- case STOPPED: return 3;
- default: return 0;
- }
+ return state.getValue();
}
private void inc(STATE state) {
@@ -75,29 +69,27 @@ public int getCount(STATE state) {
private void maybeFail(boolean fail, String action) {
if (fail) {
- throw new BrokenLifecycleEvent(action);
+ throw new BrokenLifecycleEvent(this, action);
}
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
inc(STATE.INITED);
maybeFail(failOnInit, "init");
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() {
inc(STATE.STARTED);
maybeFail(failOnStart, "start");
- super.start();
}
@Override
- public void stop() {
+ protected void serviceStop() {
inc(STATE.STOPPED);
maybeFail(failOnStop, "stop");
- super.stop();
}
public void setFailOnInit(boolean failOnInit) {
@@ -116,8 +108,13 @@ public void setFailOnStop(boolean failOnStop) {
* The exception explicitly raised on a failure
*/
public static class BrokenLifecycleEvent extends RuntimeException {
- BrokenLifecycleEvent(String action) {
- super("Lifecycle Failure during " + action);
+
+ final STATE state;
+
+ public BrokenLifecycleEvent(Service service, String action) {
+ super("Lifecycle Failure during " + action + " state is "
+ + service.getServiceState());
+ state = service.getServiceState();
}
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableStateChangeListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableStateChangeListener.java
new file mode 100644
index 0000000000..7b88c908bb
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/BreakableStateChangeListener.java
@@ -0,0 +1,102 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * A state change listener that logs the number of state change events received,
+ * and the last state invoked.
+ *
+ * It can be configured to fail during a state change event
+ */
+public class BreakableStateChangeListener
+ implements ServiceStateChangeListener {
+
+ private final String name;
+
+ private int eventCount;
+ private int failureCount;
+ private Service lastService;
+ private Service.STATE lastState = Service.STATE.NOTINITED;
+ //no callbacks are ever received for this event, so it
+ //can be used as an 'undefined'.
+ private Service.STATE failingState = Service.STATE.NOTINITED;
+ private List stateEventList = new ArrayList(4);
+
+ public BreakableStateChangeListener() {
+ this( "BreakableStateChangeListener");
+ }
+
+ public BreakableStateChangeListener(String name) {
+ this.name = name;
+ }
+
+ @Override
+ public synchronized void stateChanged(Service service) {
+ eventCount++;
+ lastService = service;
+ lastState = service.getServiceState();
+ stateEventList.add(lastState);
+ if (lastState == failingState) {
+ failureCount++;
+ throw new BreakableService.BrokenLifecycleEvent(service,
+ "Failure entering "
+ + lastState
+ + " for "
+ + service.getName());
+ }
+ }
+
+ public synchronized int getEventCount() {
+ return eventCount;
+ }
+
+ public synchronized Service getLastService() {
+ return lastService;
+ }
+
+ public synchronized Service.STATE getLastState() {
+ return lastState;
+ }
+
+ public synchronized void setFailingState(Service.STATE failingState) {
+ this.failingState = failingState;
+ }
+
+ public synchronized int getFailureCount() {
+ return failureCount;
+ }
+
+ public List getStateEventList() {
+ return stateEventList;
+ }
+
+ @Override
+ public synchronized String toString() {
+ String s =
+ name + " - event count = " + eventCount + " last state " + lastState;
+ StringBuilder history = new StringBuilder(stateEventList.size()*10);
+ for (Service.STATE state: stateEventList) {
+ history.append(state).append(" ");
+ }
+ return s + " [ " + history + "]";
+ }
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestGlobalStateChangeListener.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestGlobalStateChangeListener.java
new file mode 100644
index 0000000000..5a44bd802e
--- /dev/null
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestGlobalStateChangeListener.java
@@ -0,0 +1,220 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.service;
+
+import org.apache.hadoop.conf.Configuration;
+import org.junit.After;
+import org.junit.Test;
+
+/**
+ * Test global state changes. It is critical for all tests to clean up the
+ * global listener afterwards to avoid interfering with follow-on tests.
+ *
+ * One listener, {@link #listener} is defined which is automatically
+ * unregistered on cleanup. All other listeners must be unregistered in the
+ * finally clauses of the tests.
+ */
+public class TestGlobalStateChangeListener extends ServiceAssert {
+
+ BreakableStateChangeListener listener = new BreakableStateChangeListener("listener");
+
+
+ private void register() {
+ register(listener);
+ }
+
+ private boolean unregister() {
+ return unregister(listener);
+ }
+
+ private void register(ServiceStateChangeListener l) {
+ AbstractService.registerGlobalListener(l);
+ }
+
+ private boolean unregister(ServiceStateChangeListener l) {
+ return AbstractService.unregisterGlobalListener(l);
+ }
+
+ /**
+ * After every test case reset the list of global listeners.
+ */
+ @After
+ public void cleanup() {
+ AbstractService.resetGlobalListeners();
+ }
+
+ /**
+ * Assert that the last state of the listener is that the test expected.
+ * @param breakable a breakable listener
+ * @param state the expected state
+ */
+ public void assertListenerState(BreakableStateChangeListener breakable,
+ Service.STATE state) {
+ assertEquals("Wrong state in " + breakable, state, breakable.getLastState());
+ }
+
+ /**
+ * Assert that the number of state change notifications matches expectations.
+ * @param breakable the listener
+ * @param count the expected count.
+ */
+ public void assertListenerEventCount(BreakableStateChangeListener breakable,
+ int count) {
+ assertEquals("Wrong event count in " + breakable, count,
+ breakable.getEventCount());
+ }
+
+ /**
+ * Test that register/unregister works
+ */
+ @Test
+ public void testRegisterListener() {
+ register();
+ assertTrue("listener not registered", unregister());
+ }
+
+ /**
+ * Test that double registration results in one registration only.
+ */
+ @Test
+ public void testRegisterListenerTwice() {
+ register();
+ register();
+ assertTrue("listener not registered", unregister());
+ //there should be no listener to unregister the second time
+ assertFalse("listener double registered", unregister());
+ }
+
+ /**
+ * Test that the {@link BreakableStateChangeListener} is picking up
+ * the state changes and that its last event field is as expected.
+ */
+ @Test
+ public void testEventHistory() {
+ register();
+ BreakableService service = new BreakableService();
+ assertListenerState(listener, Service.STATE.NOTINITED);
+ assertEquals(0, listener.getEventCount());
+ service.init(new Configuration());
+ assertListenerState(listener, Service.STATE.INITED);
+ assertSame(service, listener.getLastService());
+ assertListenerEventCount(listener, 1);
+
+ service.start();
+ assertListenerState(listener, Service.STATE.STARTED);
+ assertListenerEventCount(listener, 2);
+
+ service.stop();
+ assertListenerState(listener, Service.STATE.STOPPED);
+ assertListenerEventCount(listener, 3);
+ }
+
+ /**
+ * This test triggers a failure in the listener - the expectation is that the
+ * service has already reached it's desired state, purely because the
+ * notifications take place afterwards.
+ *
+ */
+ @Test
+ public void testListenerFailure() {
+ listener.setFailingState(Service.STATE.INITED);
+ register();
+ BreakableStateChangeListener l2 = new BreakableStateChangeListener();
+ register(l2);
+ BreakableService service = new BreakableService();
+ service.init(new Configuration());
+ //expected notifications to fail
+
+ //still should record its invocation
+ assertListenerState(listener, Service.STATE.INITED);
+ assertListenerEventCount(listener, 1);
+
+ //and second listener didn't get notified of anything
+ assertListenerEventCount(l2, 0);
+
+ //service should still consider itself started
+ assertServiceStateInited(service);
+ service.start();
+ service.stop();
+ }
+
+ /**
+ * Create a chain of listeners and set one in the middle to fail; verify that
+ * those in front got called, and those after did not.
+ */
+ @Test
+ public void testListenerChain() {
+
+ //create and register the listeners
+ LoggingStateChangeListener logListener = new LoggingStateChangeListener();
+ register(logListener);
+ BreakableStateChangeListener l0 = new BreakableStateChangeListener("l0");
+ register(l0);
+ listener.setFailingState(Service.STATE.STARTED);
+ register();
+ BreakableStateChangeListener l3 = new BreakableStateChangeListener("l3");
+ register(l3);
+
+ //create and init a service.
+ BreakableService service = new BreakableService();
+ service.init(new Configuration());
+ assertServiceStateInited(service);
+ assertListenerState(l0, Service.STATE.INITED);
+ assertListenerState(listener, Service.STATE.INITED);
+ assertListenerState(l3, Service.STATE.INITED);
+
+ service.start();
+ //expect that listener l1 and the failing listener are in start, but
+ //not the final one
+ assertServiceStateStarted(service);
+ assertListenerState(l0, Service.STATE.STARTED);
+ assertListenerEventCount(l0, 2);
+ assertListenerState(listener, Service.STATE.STARTED);
+ assertListenerEventCount(listener, 2);
+ //this is the listener that is not expected to have been invoked
+ assertListenerState(l3, Service.STATE.INITED);
+ assertListenerEventCount(l3, 1);
+
+ //stop the service
+ service.stop();
+ //listeners are all updated
+ assertListenerEventCount(l0, 3);
+ assertListenerEventCount(listener, 3);
+ assertListenerEventCount(l3, 2);
+ //can all be unregistered in any order
+ unregister(logListener);
+ unregister(l0);
+ unregister(l3);
+
+ //check that the listeners are all unregistered, even
+ //though they were registered in a different order.
+ //rather than do this by doing unregister checks, a new service is created
+ service = new BreakableService();
+ //this service is initialized
+ service.init(new Configuration());
+ //it is asserted that the event count has not changed for the unregistered
+ //listeners
+ assertListenerEventCount(l0, 3);
+ assertListenerEventCount(l3, 2);
+ //except for the one listener that was not unregistered, which
+ //has incremented by one
+ assertListenerEventCount(listener, 4);
+ }
+
+}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceLifecycle.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceLifecycle.java
index c69b7b7c8e..bd73a41102 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceLifecycle.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceLifecycle.java
@@ -19,10 +19,13 @@
package org.apache.hadoop.yarn.service;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.junit.Test;
public class TestServiceLifecycle extends ServiceAssert {
+ private static Log LOG = LogFactory.getLog(TestServiceLifecycle.class);
/**
* Walk the {@link BreakableService} through it's lifecycle,
@@ -59,13 +62,8 @@ public void testInitTwice() throws Throwable {
Configuration conf = new Configuration();
conf.set("test.init","t");
svc.init(conf);
- try {
- svc.init(new Configuration());
- fail("Expected a failure, got " + svc);
- } catch (IllegalStateException e) {
- //expected
- }
- assertStateCount(svc, Service.STATE.INITED, 2);
+ svc.init(new Configuration());
+ assertStateCount(svc, Service.STATE.INITED, 1);
assertServiceConfigurationContains(svc, "test.init");
}
@@ -78,21 +76,14 @@ public void testStartTwice() throws Throwable {
BreakableService svc = new BreakableService();
svc.init(new Configuration());
svc.start();
- try {
- svc.start();
- fail("Expected a failure, got " + svc);
- } catch (IllegalStateException e) {
- //expected
- }
- assertStateCount(svc, Service.STATE.STARTED, 2);
+ svc.start();
+ assertStateCount(svc, Service.STATE.STARTED, 1);
}
/**
* Verify that when a service is stopped more than once, no exception
- * is thrown, and the counter is incremented.
- * This is because the state change operations happen after the counter in
- * the subclass is incremented, even though stop is meant to be a no-op
+ * is thrown.
* @throws Throwable if necessary
*/
@Test
@@ -103,7 +94,7 @@ public void testStopTwice() throws Throwable {
svc.stop();
assertStateCount(svc, Service.STATE.STOPPED, 1);
svc.stop();
- assertStateCount(svc, Service.STATE.STOPPED, 2);
+ assertStateCount(svc, Service.STATE.STOPPED, 1);
}
@@ -124,12 +115,12 @@ public void testStopFailedInit() throws Throwable {
//expected
}
//the service state wasn't passed
- assertServiceStateCreated(svc);
+ assertServiceStateStopped(svc);
assertStateCount(svc, Service.STATE.INITED, 1);
+ assertStateCount(svc, Service.STATE.STOPPED, 1);
//now try to stop
svc.stop();
- //even after the stop operation, we haven't entered the state
- assertServiceStateCreated(svc);
+ assertStateCount(svc, Service.STATE.STOPPED, 1);
}
@@ -151,18 +142,12 @@ public void testStopFailedStart() throws Throwable {
//expected
}
//the service state wasn't passed
- assertServiceStateInited(svc);
- assertStateCount(svc, Service.STATE.INITED, 1);
- //now try to stop
- svc.stop();
- //even after the stop operation, we haven't entered the state
- assertServiceStateInited(svc);
+ assertServiceStateStopped(svc);
}
/**
* verify that when a service fails during its stop operation,
- * its state does not change, and the subclass invocation counter
- * increments.
+ * its state does not change.
* @throws Throwable if necessary
*/
@Test
@@ -177,42 +162,302 @@ public void testFailingStop() throws Throwable {
//expected
}
assertStateCount(svc, Service.STATE.STOPPED, 1);
- assertServiceStateStarted(svc);
- //now try again, and expect it to happen again
- try {
- svc.stop();
- fail("Expected a failure, got " + svc);
- } catch (BreakableService.BrokenLifecycleEvent e) {
- //expected
- }
- assertStateCount(svc, Service.STATE.STOPPED, 2);
}
/**
- * verify that when a service that is not started is stopped, its counter
- * of stop calls is still incremented-and the service remains in its
- * original state..
+ * verify that when a service that is not started is stopped, the
+ * service enters the stopped state
* @throws Throwable on a failure
*/
@Test
public void testStopUnstarted() throws Throwable {
BreakableService svc = new BreakableService();
svc.stop();
- assertServiceStateCreated(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 1);
-
- //stop failed, now it can be initialised
- svc.init(new Configuration());
-
- //and try to stop again, with no state change but an increment
- svc.stop();
- assertServiceStateInited(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 2);
-
- //once started, the service can be stopped reliably
- svc.start();
- ServiceOperations.stop(svc);
assertServiceStateStopped(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 3);
+ assertStateCount(svc, Service.STATE.INITED, 0);
+ assertStateCount(svc, Service.STATE.STOPPED, 1);
}
+
+ /**
+ * Show that if the service failed during an init
+ * operation, stop was called.
+ */
+
+ @Test
+ public void testStopFailingInitAndStop() throws Throwable {
+ BreakableService svc = new BreakableService(true, false, true);
+ svc.register(new LoggingStateChangeListener());
+ try {
+ svc.init(new Configuration());
+ fail("Expected a failure, got " + svc);
+ } catch (BreakableService.BrokenLifecycleEvent e) {
+ assertEquals(Service.STATE.INITED, e.state);
+ }
+ //the service state is stopped
+ assertServiceStateStopped(svc);
+ assertEquals(Service.STATE.INITED, svc.getFailureState());
+
+ Throwable failureCause = svc.getFailureCause();
+ assertNotNull("Null failure cause in " + svc, failureCause);
+ BreakableService.BrokenLifecycleEvent cause =
+ (BreakableService.BrokenLifecycleEvent) failureCause;
+ assertNotNull("null state in " + cause + " raised by " + svc, cause.state);
+ assertEquals(Service.STATE.INITED, cause.state);
+ }
+
+ @Test
+ public void testInitNullConf() throws Throwable {
+ BreakableService svc = new BreakableService(false, false, false);
+ try {
+ svc.init(null);
+ LOG.warn("Null Configurations are permitted ");
+ } catch (ServiceStateException e) {
+ //expected
+ }
+ }
+
+ @Test
+ public void testServiceNotifications() throws Throwable {
+ BreakableService svc = new BreakableService(false, false, false);
+ BreakableStateChangeListener listener = new BreakableStateChangeListener();
+ svc.register(listener);
+ svc.init(new Configuration());
+ assertEventCount(listener, 1);
+ svc.start();
+ assertEventCount(listener, 2);
+ svc.stop();
+ assertEventCount(listener, 3);
+ svc.stop();
+ assertEventCount(listener, 3);
+ }
+
+ /**
+ * Test that when a service listener is unregistered, it stops being invoked
+ * @throws Throwable on a failure
+ */
+ @Test
+ public void testServiceNotificationsStopOnceUnregistered() throws Throwable {
+ BreakableService svc = new BreakableService(false, false, false);
+ BreakableStateChangeListener listener = new BreakableStateChangeListener();
+ svc.register(listener);
+ svc.init(new Configuration());
+ assertEventCount(listener, 1);
+ svc.unregister(listener);
+ svc.start();
+ assertEventCount(listener, 1);
+ svc.stop();
+ assertEventCount(listener, 1);
+ svc.stop();
+ }
+
+ /**
+ * This test uses a service listener that unregisters itself during the callbacks.
+ * This a test that verifies the concurrency logic on the listener management
+ * code, that it doesn't throw any immutable state change exceptions
+ * if you change list membership during the notifications.
+ * The standard AbstractService
implementation copies the list
+ * to an array in a synchronized
block then iterates through
+ * the copy precisely to prevent this problem.
+ * @throws Throwable on a failure
+ */
+ @Test
+ public void testServiceNotificationsUnregisterDuringCallback() throws Throwable {
+ BreakableService svc = new BreakableService(false, false, false);
+ BreakableStateChangeListener listener =
+ new SelfUnregisteringBreakableStateChangeListener();
+ BreakableStateChangeListener l2 =
+ new BreakableStateChangeListener();
+ svc.register(listener);
+ svc.register(l2);
+ svc.init(new Configuration());
+ assertEventCount(listener, 1);
+ assertEventCount(l2, 1);
+ svc.unregister(listener);
+ svc.start();
+ assertEventCount(listener, 1);
+ assertEventCount(l2, 2);
+ svc.stop();
+ assertEventCount(listener, 1);
+ svc.stop();
+ }
+
+ private static class SelfUnregisteringBreakableStateChangeListener
+ extends BreakableStateChangeListener {
+
+ @Override
+ public synchronized void stateChanged(Service service) {
+ super.stateChanged(service);
+ service.unregister(this);
+ }
+ }
+
+ private void assertEventCount(BreakableStateChangeListener listener,
+ int expected) {
+ assertEquals(listener.toString(), expected, listener.getEventCount());
+ }
+
+ @Test
+ public void testServiceFailingNotifications() throws Throwable {
+ BreakableService svc = new BreakableService(false, false, false);
+ BreakableStateChangeListener listener = new BreakableStateChangeListener();
+ listener.setFailingState(Service.STATE.STARTED);
+ svc.register(listener);
+ svc.init(new Configuration());
+ assertEventCount(listener, 1);
+ //start this; the listener failed but this won't show
+ svc.start();
+ //counter went up
+ assertEventCount(listener, 2);
+ assertEquals(1, listener.getFailureCount());
+ //stop the service -this doesn't fail
+ svc.stop();
+ assertEventCount(listener, 3);
+ assertEquals(1, listener.getFailureCount());
+ svc.stop();
+ }
+
+ /**
+ * This test verifies that you can block waiting for something to happen
+ * and use notifications to manage it
+ * @throws Throwable on a failure
+ */
+ @Test
+ public void testListenerWithNotifications() throws Throwable {
+ //this tests that a listener can get notified when a service is stopped
+ AsyncSelfTerminatingService service = new AsyncSelfTerminatingService(2000);
+ NotifyingListener listener = new NotifyingListener();
+ service.register(listener);
+ service.init(new Configuration());
+ service.start();
+ assertServiceInState(service, Service.STATE.STARTED);
+ long start = System.currentTimeMillis();
+ synchronized (listener) {
+ listener.wait(20000);
+ }
+ long duration = System.currentTimeMillis() - start;
+ assertEquals(Service.STATE.STOPPED, listener.notifyingState);
+ assertServiceInState(service, Service.STATE.STOPPED);
+ assertTrue("Duration of " + duration + " too long", duration < 10000);
+ }
+
+ @Test
+ public void testSelfTerminatingService() throws Throwable {
+ SelfTerminatingService service = new SelfTerminatingService();
+ BreakableStateChangeListener listener = new BreakableStateChangeListener();
+ service.register(listener);
+ service.init(new Configuration());
+ assertEventCount(listener, 1);
+ //start the service
+ service.start();
+ //and expect an event count of exactly two
+ assertEventCount(listener, 2);
+ }
+
+ @Test
+ public void testStartInInitService() throws Throwable {
+ Service service = new StartInInitService();
+ BreakableStateChangeListener listener = new BreakableStateChangeListener();
+ service.register(listener);
+ service.init(new Configuration());
+ assertServiceInState(service, Service.STATE.STARTED);
+ assertEventCount(listener, 1);
+ }
+
+ @Test
+ public void testStopInInitService() throws Throwable {
+ Service service = new StopInInitService();
+ BreakableStateChangeListener listener = new BreakableStateChangeListener();
+ service.register(listener);
+ service.init(new Configuration());
+ assertServiceInState(service, Service.STATE.STOPPED);
+ assertEventCount(listener, 1);
+ }
+
+ /**
+ * Listener that wakes up all threads waiting on it
+ */
+ private static class NotifyingListener implements ServiceStateChangeListener {
+ public Service.STATE notifyingState = Service.STATE.NOTINITED;
+
+ public synchronized void stateChanged(Service service) {
+ notifyingState = service.getServiceState();
+ this.notifyAll();
+ }
+ }
+
+ /**
+ * Service that terminates itself after starting and sleeping for a while
+ */
+ private static class AsyncSelfTerminatingService extends AbstractService
+ implements Runnable {
+ final int timeout;
+ private AsyncSelfTerminatingService(int timeout) {
+ super("AsyncSelfTerminatingService");
+ this.timeout = timeout;
+ }
+
+ @Override
+ protected void serviceStart() throws Exception {
+ new Thread(this).start();
+ super.serviceStart();
+ }
+
+ @Override
+ public void run() {
+ try {
+ Thread.sleep(timeout);
+ } catch (InterruptedException ignored) {
+
+ }
+ this.stop();
+ }
+ }
+
+ /**
+ * Service that terminates itself in startup
+ */
+ private static class SelfTerminatingService extends AbstractService {
+ private SelfTerminatingService() {
+ super("SelfTerminatingService");
+ }
+
+ @Override
+ protected void serviceStart() throws Exception {
+ //start
+ super.serviceStart();
+ //then stop
+ stop();
+ }
+ }
+
+ /**
+ * Service that starts itself in init
+ */
+ private static class StartInInitService extends AbstractService {
+ private StartInInitService() {
+ super("StartInInitService");
+ }
+
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
+ start();
+ }
+ }
+
+ /**
+ * Service that starts itself in init
+ */
+ private static class StopInInitService extends AbstractService {
+ private StopInInitService() {
+ super("StopInInitService");
+ }
+
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
+ stop();
+ }
+ }
+
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceOperations.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceOperations.java
deleted file mode 100644
index 14aa1f522c..0000000000
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/service/TestServiceOperations.java
+++ /dev/null
@@ -1,312 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-
-package org.apache.hadoop.yarn.service;
-
-import org.apache.hadoop.conf.Configuration;
-import org.junit.Test;
-
-/**
- * These tests verify that the {@link ServiceOperations} methods
- * do a best-effort attempt to make the service state change operations
- * idempotent. That is still best effort -there is no thread safety, and
- * a failure during a state change does not prevent the operation
- * being called again.
- */
-public class TestServiceOperations extends ServiceAssert {
-
- @Test
- public void testWalkthrough() throws Throwable {
- BreakableService svc = new BreakableService();
- assertServiceStateCreated(svc);
- Configuration conf = new Configuration();
- conf.set("test.walkthrough","t");
- ServiceOperations.init(svc, conf);
- assertServiceStateInited(svc);
- assertStateCount(svc, Service.STATE.INITED, 1);
- //check the configuration made it all the way through.
- assertServiceConfigurationContains(svc, "test.walkthrough");
- ServiceOperations.start(svc);
- assertServiceStateStarted(svc);
- assertStateCount(svc, Service.STATE.STARTED, 1);
- ServiceOperations.stop(svc);
- assertServiceStateStopped(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 1);
- }
-
- /**
- * Call init twice -expect a failure, and expect the count
- * of initialization attempts to still be 1: the state
- * check was made before the subclass method was called.
- * @throws Throwable if need be
- */
- @Test
- public void testInitTwice() throws Throwable {
- BreakableService svc = new BreakableService();
- Configuration conf = new Configuration();
- conf.set("test.init", "t");
- ServiceOperations.init(svc, conf);
- try {
- ServiceOperations.init(svc, new Configuration());
- fail("Expected a failure, got " + svc);
- } catch (IllegalStateException e) {
- //expected
- }
- assertStateCount(svc, Service.STATE.INITED, 1);
- assertServiceConfigurationContains(svc, "test.init");
- }
-
- /**
- * call start twice; expect failures and the start invoke count to
- * be exactly 1.
- * @throws Throwable if necessary
- */
- @Test
- public void testStartTwice() throws Throwable {
- BreakableService svc = new BreakableService();
- ServiceOperations.init(svc, new Configuration());
- ServiceOperations.start(svc);
- try {
- ServiceOperations.start(svc);
- fail("Expected a failure, got " + svc);
- } catch (IllegalStateException e) {
- //expected
- }
- assertStateCount(svc, Service.STATE.STARTED, 1);
- }
-
- /**
- * Test that the deploy operation pushes a service into its started state
- * @throws Throwable on any failure.
- */
- @Test
- public void testDeploy() throws Throwable {
- BreakableService svc = new BreakableService();
- assertServiceStateCreated(svc);
- ServiceOperations.deploy(svc, new Configuration());
- assertServiceStateStarted(svc);
- assertStateCount(svc, Service.STATE.INITED, 1);
- assertStateCount(svc, Service.STATE.STARTED, 1);
- ServiceOperations.stop(svc);
- assertServiceStateStopped(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 1);
- }
-
- /**
- * Demonstrate that the deploy operation fails when invoked twice,
- * but the service method call counts are unchanged after the second call.
- * @throws Throwable on any failure.
- */
- @Test
- public void testDeployNotIdempotent() throws Throwable {
- BreakableService svc = new BreakableService();
- assertServiceStateCreated(svc);
- ServiceOperations.deploy(svc, new Configuration());
- try {
- ServiceOperations.deploy(svc, new Configuration());
- fail("Expected a failure, got " + svc);
- } catch (IllegalStateException e) {
- //expected
- }
- //verify state and values are unchanged
- assertServiceStateStarted(svc);
- assertStateCount(svc, Service.STATE.INITED, 1);
- assertStateCount(svc, Service.STATE.STARTED, 1);
- ServiceOperations.stop(svc);
- }
-
- /**
- * Test that the deploy operation can fail part way through, in which
- * case the service is in the state that it was in before the failing
- * state method was called.
- * @throws Throwable on any failure.
- */
- @Test
- public void testDeployNotAtomic() throws Throwable {
- //this instance is set to fail in the start() call.
- BreakableService svc = new BreakableService(false, true, false);
- try {
- ServiceOperations.deploy(svc, new Configuration());
- fail("Expected a failure, got " + svc);
- } catch (BreakableService.BrokenLifecycleEvent expected) {
- //expected
- }
- //now in the inited state
- assertServiceStateInited(svc);
- assertStateCount(svc, Service.STATE.INITED, 1);
- assertStateCount(svc, Service.STATE.STARTED, 1);
- //try again -expect a failure as the service is now inited.
- try {
- ServiceOperations.deploy(svc, new Configuration());
- fail("Expected a failure, got " + svc);
- } catch (IllegalStateException e) {
- //expected
- }
- }
-
- /**
- * verify that when a service is stopped more than once, no exception
- * is thrown, and the counter is not incremented
- * this is because the state change operations happen after the counter in
- * the subclass is incremented, even though stop is meant to be a no-op
- * @throws Throwable on a failure
- */
- @Test
- public void testStopTwice() throws Throwable {
- BreakableService svc = new BreakableService();
- ServiceOperations.deploy(svc, new Configuration());
- ServiceOperations.stop(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 1);
- assertServiceStateStopped(svc);
- ServiceOperations.stop(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 1);
- }
-
- /**
- * verify that when a service that is not started is stopped, it's counter
- * is not incremented -the stop() method was not invoked.
- * @throws Throwable on a failure
- */
- @Test
- public void testStopInit() throws Throwable {
- BreakableService svc = new BreakableService();
- ServiceOperations.stop(svc);
- assertServiceStateCreated(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 0);
- ServiceOperations.stop(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 0);
- }
-
-
- /**
- * Show that if the service failed during an init
- * operation, it stays in the created state, even after stopping it
- * @throws Throwable
- */
-
- @Test
- public void testStopFailedInit() throws Throwable {
- BreakableService svc = new BreakableService(true, false, false);
- assertServiceStateCreated(svc);
- try {
- ServiceOperations.init(svc, new Configuration());
- fail("Expected a failure, got " + svc);
- } catch (BreakableService.BrokenLifecycleEvent e) {
- //expected
- }
- //the service state wasn't passed
- assertServiceStateCreated(svc);
- //the init state got invoked once
- assertStateCount(svc, Service.STATE.INITED, 1);
- //now try to stop
- ServiceOperations.stop(svc);
- //even after the stop operation, we haven't entered the state
- assertServiceStateCreated(svc);
- }
-
-
- /**
- * Show that if the service failed during an init
- * operation, it stays in the created state, even after stopping it
- * @throws Throwable
- */
-
- @Test
- public void testStopFailedStart() throws Throwable {
- BreakableService svc = new BreakableService(false, true, false);
- ServiceOperations.init(svc, new Configuration());
- assertServiceStateInited(svc);
- try {
- ServiceOperations.start(svc);
- fail("Expected a failure, got " + svc);
- } catch (BreakableService.BrokenLifecycleEvent e) {
- //expected
- }
- //the service state wasn't passed
- assertServiceStateInited(svc);
- assertStateCount(svc, Service.STATE.INITED, 1);
- //now try to stop
- ServiceOperations.stop(svc);
- //even after the stop operation, we haven't entered the state
- assertServiceStateInited(svc);
- }
-
- /**
- * verify that when a service is stopped more than once, no exception
- * is thrown, and the counter is incremented
- * this is because the state change operations happen after the counter in
- * the subclass is incremented, even though stop is meant to be a no-op.
- *
- * The {@link ServiceOperations#stop(Service)} operation does not prevent
- * this from happening
- * @throws Throwable
- */
- @Test
- public void testFailingStop() throws Throwable {
- BreakableService svc = new BreakableService(false, false, true);
- ServiceOperations.deploy(svc, new Configuration());
- try {
- ServiceOperations.stop(svc);
- fail("Expected a failure, got " + svc);
- } catch (BreakableService.BrokenLifecycleEvent e) {
- //expected
- }
- assertStateCount(svc, Service.STATE.STOPPED, 1);
- //now try to stop, this time doing it quietly
- Exception exception = ServiceOperations.stopQuietly(svc);
- assertTrue("Wrong exception type : " + exception,
- exception instanceof BreakableService.BrokenLifecycleEvent);
- assertStateCount(svc, Service.STATE.STOPPED, 2);
- }
-
-
- /**
- * verify that when a service that is not started is stopped, its counter
- * of stop calls is still incremented-and the service remains in its
- * original state..
- * @throws Throwable on a failure
- */
- @Test
- public void testStopUnstarted() throws Throwable {
- BreakableService svc = new BreakableService();
-
- //invocation in NOTINITED state should be no-op
- ServiceOperations.stop(svc);
- assertServiceStateCreated(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 0);
-
- //stop failed, now it can be initialised
- ServiceOperations.init(svc, new Configuration());
-
- //again, no-op
- ServiceOperations.stop(svc);
- assertServiceStateInited(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 0);
-
- //once started, the service can be stopped reliably
- ServiceOperations.start(svc);
- ServiceOperations.stop(svc);
- assertServiceStateStopped(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 1);
-
- //now stop one more time
- ServiceOperations.stop(svc);
- assertStateCount(svc, Service.STATE.STOPPED, 1);
- }
-}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestCompositeService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestCompositeService.java
index 0fc598ad7f..753d35f0fb 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestCompositeService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/util/TestCompositeService.java
@@ -21,10 +21,15 @@
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.fail;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.YarnRuntimeException;
+import org.apache.hadoop.yarn.service.BreakableService;
import org.apache.hadoop.yarn.service.CompositeService;
+import org.apache.hadoop.yarn.service.Service;
import org.apache.hadoop.yarn.service.Service.STATE;
+import org.apache.hadoop.yarn.service.ServiceStateException;
import org.junit.Before;
import org.junit.Test;
@@ -34,6 +39,16 @@ public class TestCompositeService {
private static final int FAILED_SERVICE_SEQ_NUMBER = 2;
+ private static final Log LOG = LogFactory.getLog(TestCompositeService.class);
+
+ /**
+ * flag to state policy of CompositeService, and hence
+ * what to look for after trying to stop a service from another state
+ * (e.g inited)
+ */
+ private static final boolean STOP_ONLY_STARTED_SERVICES =
+ CompositeServiceImpl.isPolicyToStopOnlyStartedServices();
+
@Before
public void setup() {
CompositeServiceImpl.resetCounter();
@@ -59,6 +74,9 @@ public void testCallSequence() {
// Initialise the composite service
serviceManager.init(conf);
+ //verify they were all inited
+ assertInState(STATE.INITED, services);
+
// Verify the init() call sequence numbers for every service
for (int i = 0; i < NUM_OF_SERVICES; i++) {
assertEquals("For " + services[i]
@@ -67,11 +85,11 @@ public void testCallSequence() {
}
// Reset the call sequence numbers
- for (int i = 0; i < NUM_OF_SERVICES; i++) {
- services[i].reset();
- }
+ resetServices(services);
serviceManager.start();
+ //verify they were all started
+ assertInState(STATE.STARTED, services);
// Verify the start() call sequence numbers for every service
for (int i = 0; i < NUM_OF_SERVICES; i++) {
@@ -79,13 +97,12 @@ public void testCallSequence() {
+ " service, start() call sequence number should have been ", i,
services[i].getCallSequenceNumber());
}
+ resetServices(services);
- // Reset the call sequence numbers
- for (int i = 0; i < NUM_OF_SERVICES; i++) {
- services[i].reset();
- }
serviceManager.stop();
+ //verify they were all stopped
+ assertInState(STATE.STOPPED, services);
// Verify the stop() call sequence numbers for every service
for (int i = 0; i < NUM_OF_SERVICES; i++) {
@@ -104,6 +121,13 @@ public void testCallSequence() {
}
}
+ private void resetServices(CompositeServiceImpl[] services) {
+ // Reset the call sequence numbers
+ for (int i = 0; i < NUM_OF_SERVICES; i++) {
+ services[i].reset();
+ }
+ }
+
@Test
public void testServiceStartup() {
ServiceManager serviceManager = new ServiceManager("ServiceManager");
@@ -131,7 +155,7 @@ public void testServiceStartup() {
fail("Exception should have been thrown due to startup failure of last service");
} catch (YarnRuntimeException e) {
for (int i = 0; i < NUM_OF_SERVICES - 1; i++) {
- if (i >= FAILED_SERVICE_SEQ_NUMBER) {
+ if (i >= FAILED_SERVICE_SEQ_NUMBER && STOP_ONLY_STARTED_SERVICES) {
// Failed service state should be INITED
assertEquals("Service state should have been ", STATE.INITED,
services[NUM_OF_SERVICES - 1].getServiceState());
@@ -171,15 +195,147 @@ public void testServiceStop() {
try {
serviceManager.stop();
} catch (YarnRuntimeException e) {
- for (int i = 0; i < NUM_OF_SERVICES - 1; i++) {
- assertEquals("Service state should have been ", STATE.STOPPED,
- services[NUM_OF_SERVICES].getServiceState());
- }
+ }
+ assertInState(STATE.STOPPED, services);
+ }
+
+ /**
+ * Assert that all services are in the same expected state
+ * @param expected expected state value
+ * @param services services to examine
+ */
+ private void assertInState(STATE expected, CompositeServiceImpl[] services) {
+ assertInState(expected, services,0, services.length);
+ }
+
+ /**
+ * Assert that all services are in the same expected state
+ * @param expected expected state value
+ * @param services services to examine
+ * @param start start offset
+ * @param finish finish offset: the count stops before this number
+ */
+ private void assertInState(STATE expected,
+ CompositeServiceImpl[] services,
+ int start, int finish) {
+ for (int i = start; i < finish; i++) {
+ Service service = services[i];
+ assertInState(expected, service);
}
}
+ private void assertInState(STATE expected, Service service) {
+ assertEquals("Service state should have been " + expected + " in "
+ + service,
+ expected,
+ service.getServiceState());
+ }
+
+ /**
+ * Shut down from not-inited: expect nothing to have happened
+ */
+ @Test
+ public void testServiceStopFromNotInited() {
+ ServiceManager serviceManager = new ServiceManager("ServiceManager");
+
+ // Add services
+ for (int i = 0; i < NUM_OF_SERVICES; i++) {
+ CompositeServiceImpl service = new CompositeServiceImpl(i);
+ serviceManager.addTestService(service);
+ }
+
+ CompositeServiceImpl[] services = serviceManager.getServices().toArray(
+ new CompositeServiceImpl[0]);
+ serviceManager.stop();
+ assertInState(STATE.NOTINITED, services);
+ }
+
+ /**
+ * Shut down from inited
+ */
+ @Test
+ public void testServiceStopFromInited() {
+ ServiceManager serviceManager = new ServiceManager("ServiceManager");
+
+ // Add services
+ for (int i = 0; i < NUM_OF_SERVICES; i++) {
+ CompositeServiceImpl service = new CompositeServiceImpl(i);
+ serviceManager.addTestService(service);
+ }
+
+ CompositeServiceImpl[] services = serviceManager.getServices().toArray(
+ new CompositeServiceImpl[0]);
+ serviceManager.init(new Configuration());
+ serviceManager.stop();
+ if (STOP_ONLY_STARTED_SERVICES) {
+ //this policy => no services were stopped
+ assertInState(STATE.INITED, services);
+ } else {
+ assertInState(STATE.STOPPED, services);
+ }
+ }
+
+ /**
+ * Use a null configuration & expect a failure
+ * @throws Throwable
+ */
+ @Test
+ public void testInitNullConf() throws Throwable {
+ ServiceManager serviceManager = new ServiceManager("testInitNullConf");
+
+ CompositeServiceImpl service = new CompositeServiceImpl(0);
+ serviceManager.addTestService(service);
+ try {
+ serviceManager.init(null);
+ LOG.warn("Null Configurations are permitted " + serviceManager);
+ } catch (ServiceStateException e) {
+ //expected
+ }
+ }
+
+ /**
+ * Walk the service through their lifecycle without any children;
+ * verify that it all works.
+ */
+ @Test
+ public void testServiceLifecycleNoChildren() {
+ ServiceManager serviceManager = new ServiceManager("ServiceManager");
+ serviceManager.init(new Configuration());
+ serviceManager.start();
+ serviceManager.stop();
+ }
+
+ @Test
+ public void testAddServiceInInit() throws Throwable {
+ BreakableService child = new BreakableService();
+ assertInState(STATE.NOTINITED, child);
+ CompositeServiceAddingAChild composite =
+ new CompositeServiceAddingAChild(child);
+ composite.init(new Configuration());
+ assertInState(STATE.INITED, child);
+ }
+
+ public static class CompositeServiceAddingAChild extends CompositeService{
+ Service child;
+
+ public CompositeServiceAddingAChild(Service child) {
+ super("CompositeServiceAddingAChild");
+ this.child = child;
+ }
+
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
+ addService(child);
+ super.serviceInit(conf);
+ }
+ }
+
public static class CompositeServiceImpl extends CompositeService {
+ public static boolean isPolicyToStopOnlyStartedServices() {
+ return STOP_ONLY_STARTED_SERVICES;
+ }
+
private static int counter = -1;
private int callSequenceNumber = -1;
@@ -193,30 +349,30 @@ public CompositeServiceImpl(int sequenceNumber) {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
counter++;
callSequenceNumber = counter;
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
if (throwExceptionOnStart) {
throw new YarnRuntimeException("Fake service start exception");
}
counter++;
callSequenceNumber = counter;
- super.start();
+ super.serviceStart();
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
counter++;
callSequenceNumber = counter;
if (throwExceptionOnStop) {
throw new YarnRuntimeException("Fake service stop exception");
}
- super.stop();
+ super.serviceStop();
}
public static int getCounter() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
index 4ce630a951..a967daa13c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/DeletionService.java
@@ -30,7 +30,6 @@
import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
-import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.service.AbstractService;
import org.apache.commons.logging.Log;
@@ -75,7 +74,7 @@ public void delete(String user, Path subDir, Path... baseDirs) {
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
ThreadFactory tf = new ThreadFactoryBuilder()
.setNameFormat("DeletionService #%d")
.build();
@@ -90,21 +89,23 @@ public void init(Configuration conf) {
}
sched.setExecuteExistingDelayedTasksAfterShutdownPolicy(false);
sched.setKeepAliveTime(60L, SECONDS);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void stop() {
- sched.shutdown();
- boolean terminated = false;
- try {
- terminated = sched.awaitTermination(10, SECONDS);
- } catch (InterruptedException e) {
+ protected void serviceStop() throws Exception {
+ if (sched != null) {
+ sched.shutdown();
+ boolean terminated = false;
+ try {
+ terminated = sched.awaitTermination(10, SECONDS);
+ } catch (InterruptedException e) {
+ }
+ if (terminated != true) {
+ sched.shutdownNow();
+ }
}
- if (terminated != true) {
- sched.shutdownNow();
- }
- super.stop();
+ super.serviceStop();
}
/**
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
index 9b66fa44c9..da977a4b4e 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LocalDirsHandlerService.java
@@ -113,7 +113,7 @@ public LocalDirsHandlerService() {
*
*/
@Override
- public void init(Configuration config) {
+ protected void serviceInit(Configuration config) throws Exception {
// Clone the configuration as we may do modifications to dirs-list
Configuration conf = new Configuration(config);
diskHealthCheckInterval = conf.getLong(
@@ -126,7 +126,7 @@ public void init(Configuration config) {
YarnConfiguration.NM_MIN_HEALTHY_DISKS_FRACTION,
YarnConfiguration.DEFAULT_NM_MIN_HEALTHY_DISKS_FRACTION);
lastDisksCheckTime = System.currentTimeMillis();
- super.init(conf);
+ super.serviceInit(conf);
FileContext localFs;
try {
@@ -150,24 +150,24 @@ public void init(Configuration config) {
* Method used to start the disk health monitoring, if enabled.
*/
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
if (isDiskHealthCheckerEnabled) {
dirsHandlerScheduler = new Timer("DiskHealthMonitor-Timer", true);
dirsHandlerScheduler.scheduleAtFixedRate(monitoringTimerTask,
diskHealthCheckInterval, diskHealthCheckInterval);
}
- super.start();
+ super.serviceStart();
}
/**
* Method used to terminate the disk health monitoring service.
*/
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (dirsHandlerScheduler != null) {
dirsHandlerScheduler.cancel();
}
- super.stop();
+ super.serviceStop();
}
/**
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java
index 78e5a53685..d3ad730187 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthCheckerService.java
@@ -39,13 +39,13 @@ public NodeHealthCheckerService() {
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
if (NodeHealthScriptRunner.shouldRun(conf)) {
nodeHealthScriptRunner = new NodeHealthScriptRunner();
addService(nodeHealthScriptRunner);
}
addService(dirsHandler);
- super.init(conf);
+ super.serviceInit(conf);
}
/**
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthScriptRunner.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthScriptRunner.java
index ca3dc3c8aa..bc1a5dd677 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthScriptRunner.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeHealthScriptRunner.java
@@ -197,7 +197,7 @@ public NodeHealthScriptRunner() {
* Method which initializes the values for the script path and interval time.
*/
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
this.conf = conf;
this.nodeHealthScript =
conf.get(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_PATH);
@@ -209,6 +209,7 @@ public void init(Configuration conf) {
String[] args = conf.getStrings(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_OPTS,
new String[] {});
timer = new NodeHealthMonitorExecutor(args);
+ super.serviceInit(conf);
}
/**
@@ -216,7 +217,7 @@ public void init(Configuration conf) {
*
*/
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
// if health script path is not configured don't start the thread.
if (!shouldRun(conf)) {
LOG.info("Not starting node health monitor");
@@ -226,6 +227,7 @@ public void start() {
// Start the timer task immediately and
// then periodically at interval time.
nodeHealthScriptScheduler.scheduleAtFixedRate(timer, 0, intervalTime);
+ super.serviceStart();
}
/**
@@ -233,11 +235,13 @@ public void start() {
*
*/
@Override
- public void stop() {
+ protected void serviceStop() {
if (!shouldRun(conf)) {
return;
}
- nodeHealthScriptScheduler.cancel();
+ if (nodeHealthScriptScheduler != null) {
+ nodeHealthScriptScheduler.cancel();
+ }
if (shexec != null) {
Process p = shexec.getProcess();
if (p != null) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
index 04bf9ee118..dbe800455c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeManager.java
@@ -128,7 +128,7 @@ protected void doSecureLogin() throws IOException {
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
conf.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
@@ -192,31 +192,36 @@ public void init(Configuration conf) {
YarnConfiguration.DEFAULT_NM_PROCESS_KILL_WAIT_MS) +
SHUTDOWN_CLEANUP_SLOP_MS;
- super.init(conf);
+ super.serviceInit(conf);
// TODO add local dirs to del
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
try {
doSecureLogin();
} catch (IOException e) {
throw new YarnRuntimeException("Failed NodeManager login", e);
}
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (isStopping.getAndSet(true)) {
return;
}
-
- cleanupContainers(NodeManagerEventType.SHUTDOWN);
- super.stop();
+ if (context != null) {
+ cleanupContainers(NodeManagerEventType.SHUTDOWN);
+ }
+ super.serviceStop();
DefaultMetricsSystem.shutdown();
}
+ public String getName() {
+ return "NodeManager";
+ }
+
protected void resyncWithRM() {
//we do not want to block dispatcher thread here
new Thread() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
index 9c9994ba0c..9036b11bff 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/NodeStatusUpdaterImpl.java
@@ -80,7 +80,7 @@ public class NodeStatusUpdaterImpl extends AbstractService implements
private InetSocketAddress rmAddress;
private Resource totalResource;
private int httpPort;
- private boolean isStopped;
+ private volatile boolean isStopped;
private RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
private boolean tokenKeepAliveEnabled;
private long tokenRemovalDelayMs;
@@ -109,7 +109,7 @@ public NodeStatusUpdaterImpl(Context context, Dispatcher dispatcher,
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
this.rmAddress = conf.getSocketAddr(
YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
@@ -146,11 +146,11 @@ public synchronized void init(Configuration conf) {
" physical-memory=" + memoryMb + " virtual-memory=" + virtualMemoryMb +
" physical-cores=" + cpuCores + " virtual-cores=" + virtualCores);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
// NodeManager is the last service to start, so NodeId is available.
this.nodeId = this.context.getNodeId();
@@ -159,7 +159,7 @@ public void start() {
// Registration has to be in start so that ContainerManager can get the
// perNM tokens needed to authenticate ContainerTokens.
registerWithRM();
- super.start();
+ super.serviceStart();
startStatusUpdater();
} catch (Exception e) {
String errorMessage = "Unexpected error starting NodeStatusUpdater";
@@ -169,10 +169,10 @@ public void start() {
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
// Interrupt the updater.
this.isStopped = true;
- super.stop();
+ super.serviceStop();
}
protected void rebootNodeStatusUpdater() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
index 83676e57ed..e20f44ded2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/AuxServices.java
@@ -30,7 +30,6 @@
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.service.AbstractService;
@@ -82,7 +81,7 @@ public Map getMeta() {
}
@Override
- public void init(Configuration conf) {
+ public void serviceInit(Configuration conf) throws Exception {
Collection auxNames = conf.getStringCollection(
YarnConfiguration.NM_AUX_SERVICES);
for (final String sName : auxNames) {
@@ -110,11 +109,11 @@ public void init(Configuration conf) {
throw e;
}
}
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ public void serviceStart() throws Exception {
// TODO fork(?) services running as configured user
// monitor for health, shutdown/restart(?) if any should die
for (Map.Entry entry : serviceMap.entrySet()) {
@@ -127,11 +126,11 @@ public void start() {
serviceMeta.put(name, meta);
}
}
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ public void serviceStop() throws Exception {
try {
synchronized (serviceMap) {
for (Service service : serviceMap.values()) {
@@ -144,7 +143,7 @@ public void stop() {
serviceMeta.clear();
}
} finally {
- super.stop();
+ super.serviceStop();
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
index 71de5fcafa..6a33bf2d79 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
@@ -177,13 +177,13 @@ public ContainerManagerImpl(Context context, ContainerExecutor exec,
}
@Override
- public void init(Configuration conf) {
+ public void serviceInit(Configuration conf) throws Exception {
LogHandler logHandler =
createLogHandler(conf, this.context, this.deletionService);
addIfService(logHandler);
dispatcher.register(LogHandlerEventType.class, logHandler);
- super.init(conf);
+ super.serviceInit(conf);
}
private void addIfService(Object object) {
@@ -220,7 +220,7 @@ protected ContainersLauncher createContainersLauncher(Context context,
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
// Enqueue user dirs in deletion context
@@ -254,7 +254,7 @@ public void start() {
connectAddress.getPort());
((NodeManager.NMContext)context).setNodeId(nodeId);
LOG.info("ContainerManager started at " + connectAddress);
- super.start();
+ super.serviceStart();
}
void refreshServiceAcls(Configuration configuration,
@@ -263,14 +263,14 @@ void refreshServiceAcls(Configuration configuration,
}
@Override
- public void stop() {
+ public void serviceStop() throws Exception {
if (auxiliaryServices.getServiceState() == STARTED) {
auxiliaryServices.unregister(this);
}
if (server != null) {
server.stop();
}
- super.stop();
+ super.serviceStop();
}
// Get the remoteUGI corresponding to the api call.
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
index 163b2dcaa9..5aeb8cde6f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainersLauncher.java
@@ -91,20 +91,20 @@ public ContainersLauncher(Context context, Dispatcher dispatcher,
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
try {
//TODO Is this required?
FileContext.getLocalFSFileContext(conf);
} catch (UnsupportedFileSystemException e) {
throw new YarnRuntimeException("Failed to start ContainersLauncher", e);
}
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
containerLauncher.shutdownNow();
- super.stop();
+ super.serviceStop();
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
index eb79c50bf4..dcc45c8d97 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ResourceLocalizationService.java
@@ -194,7 +194,7 @@ private void validateConf(Configuration conf) {
}
@Override
- public void init(Configuration conf) {
+ public void serviceInit(Configuration conf) throws Exception {
this.validateConf(conf);
this.publicRsrc =
new LocalResourcesTrackerImpl(null, dispatcher, true, conf);
@@ -239,7 +239,7 @@ public void init(Configuration conf) {
localizerTracker = createLocalizerTracker(conf);
addService(localizerTracker);
dispatcher.register(LocalizerEventType.class, localizerTracker);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
@@ -248,7 +248,7 @@ public LocalizerHeartbeatResponse heartbeat(LocalizerStatus status) {
}
@Override
- public void start() {
+ public void serviceStart() throws Exception {
cacheCleanup.scheduleWithFixedDelay(new CacheCleanup(dispatcher),
cacheCleanupPeriod, cacheCleanupPeriod, TimeUnit.MILLISECONDS);
server = createServer();
@@ -257,7 +257,7 @@ public void start() {
getConfig().updateConnectAddr(YarnConfiguration.NM_LOCALIZER_ADDRESS,
server.getListenerAddress());
LOG.info("Localizer started on port " + server.getPort());
- super.start();
+ super.serviceStart();
}
LocalizerTracker createLocalizerTracker(Configuration conf) {
@@ -288,12 +288,12 @@ Server createServer() {
}
@Override
- public void stop() {
+ public void serviceStop() throws Exception {
if (server != null) {
server.stop();
}
cacheCleanup.shutdown();
- super.stop();
+ super.serviceStop();
}
@Override
@@ -536,9 +536,9 @@ class LocalizerTracker extends AbstractService implements EventHandler= connectionWaitSecs * 1000)
+ && (duration < (connectionWaitSecs * 1000 + delta));
+ if(!waitTimeValid) {
+ //either the exception was too early, or it had a different cause.
+ //reject with the inner stack trace
+ throw new Exception("NM should have tried re-connecting to RM during " +
"period of at least " + connectionWaitSecs + " seconds, but " +
"stopped retrying within " + (connectionWaitSecs + delta/1000) +
- " seconds", (System.currentTimeMillis() - waitStartTime
- >= connectionWaitSecs*1000) && (System.currentTimeMillis()
- - waitStartTime < (connectionWaitSecs*1000+delta)));
+ " seconds: " + e, e);
+ }
}
//Test NM connect to RM, fail at first several attempts,
//but finally success.
- nm = new NodeManager() {
+ nm = nmWithUpdater = new NodeManagerWithCustomNodeStatusUpdater() {
@Override
- protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+ protected NodeStatusUpdater createUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
NodeStatusUpdater nodeStatusUpdater = new MyNodeStatusUpdater4(
context, dispatcher, healthChecker, metrics, rmStartIntervalMS,
@@ -822,20 +888,33 @@ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
return nodeStatusUpdater;
}
};
-
nm.init(conf);
+ NodeStatusUpdater updater = nmWithUpdater.getUpdater();
+ Assert.assertNotNull("Updater not yet created ", updater);
waitStartTime = System.currentTimeMillis();
try {
nm.start();
} catch (Exception ex){
- Assert.fail("NM should have started successfully " +
- "after connecting to RM.");
+ LOG.error("NM should have started successfully " +
+ "after connecting to RM.", ex);
+ throw ex;
}
- Assert.assertTrue("NM should have connected to RM within " + delta/1000
- +" seconds of RM starting up.",
- (System.currentTimeMillis() - waitStartTime >= rmStartIntervalMS)
- && (System.currentTimeMillis() - waitStartTime
- < (rmStartIntervalMS+delta)));
+ long duration = System.currentTimeMillis() - waitStartTime;
+ MyNodeStatusUpdater4 myUpdater = (MyNodeStatusUpdater4) updater;
+ Assert.assertTrue("Updater was never started",
+ myUpdater.getWaitStartTime()>0);
+ Assert.assertTrue("NM started before updater triggered",
+ myUpdater.isTriggered());
+ Assert.assertTrue("NM should have connected to RM after "
+ +"the start interval of " + rmStartIntervalMS
+ +": actual " + duration
+ + " " + myUpdater,
+ (duration >= rmStartIntervalMS));
+ Assert.assertTrue("NM should have connected to RM less than "
+ + (rmStartIntervalMS + delta)
+ +" milliseconds of RM starting up: actual " + duration
+ + " " + myUpdater,
+ (duration < (rmStartIntervalMS + delta)));
}
/**
@@ -846,7 +925,7 @@ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
* only after NM_EXPIRY interval. See MAPREDUCE-2749.
*/
@Test
- public void testNoRegistrationWhenNMServicesFail() {
+ public void testNoRegistrationWhenNMServicesFail() throws Exception {
nm = new NodeManager() {
@Override
@@ -865,7 +944,7 @@ protected ContainerManagerImpl createContainerManager(Context context,
return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
metrics, aclsManager, diskhandler) {
@Override
- public void start() {
+ protected void serviceStart() {
// Simulating failure of starting RPC server
throw new YarnRuntimeException("Starting of RPC Server failed");
}
@@ -961,7 +1040,7 @@ public void testNodeStatusUpdaterRetryAndNMShutdown()
nm.init(conf);
nm.start();
try {
- syncBarrier.await();
+ syncBarrier.await(10000, TimeUnit.MILLISECONDS);
} catch (Exception e) {
}
Assert.assertTrue(((MyNodeManager2) nm).isStopped);
@@ -1053,20 +1132,25 @@ private Container getMockContainer(ContainerStatus containerStatus) {
}
}
- private void verifyNodeStartFailure(String errMessage) {
+ private void verifyNodeStartFailure(String errMessage) throws Exception {
+ Assert.assertNotNull("nm is null", nm);
YarnConfiguration conf = createNMConfig();
nm.init(conf);
try {
nm.start();
Assert.fail("NM should have failed to start. Didn't get exception!!");
} catch (Exception e) {
- Assert.assertEquals(errMessage, e.getCause()
- .getMessage());
+ //the version in trunk looked in the cause for equality
+ // and assumed failures were nested.
+ //this version assumes that error strings propagate to the base and
+ //use a contains() test only. It should be less brittle
+ if(!e.getMessage().contains(errMessage)) {
+ throw e;
+ }
}
- // the state change to stopped occurs only if the startup is success, else
- // state change doesn't occur
- Assert.assertEquals("NM state is wrong!", Service.STATE.INITED, nm
+ // the service should be stopped
+ Assert.assertEquals("NM state is wrong!", STATE.STOPPED, nm
.getServiceState());
Assert.assertEquals("Number of registered nodes is wrong!", 0,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
index 6aa4ff9316..af6f582100 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
@@ -202,8 +202,7 @@ public void delete(String user, Path subDir, Path[] baseDirs) {
@After
public void tearDown() throws IOException, InterruptedException {
- if (containerManager != null
- && containerManager.getServiceState() == STATE.STARTED) {
+ if (containerManager != null) {
containerManager.stop();
}
createContainerExecutor().deleteAsUser(user,
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
index 38ced35a26..ad7049feb1 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
@@ -63,20 +63,20 @@ static class LightService extends AbstractService
}
public ArrayList getAppIdsStopped() {
- return (ArrayList)this.stoppedApps.clone();
+ return (ArrayList)this.stoppedApps.clone();
}
- @Override
- public void init(Configuration conf) {
+ @Override
+ protected void serviceInit(Configuration conf) throws Exception {
remaining_init = conf.getInt(idef + ".expected.init", 0);
remaining_stop = conf.getInt(idef + ".expected.stop", 0);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
assertEquals(0, remaining_init);
assertEquals(0, remaining_stop);
- super.stop();
+ super.serviceStop();
}
@Override
public void initApp(String user, ApplicationId appId, ByteBuffer data) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
index 8728beeb91..d4832e0c98 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
@@ -237,7 +237,7 @@ public void testResourceRelease() throws Exception {
dirsHandler.init(conf);
DeletionService delService = new DeletionService(exec);
- delService.init(null);
+ delService.init(new Configuration());
delService.start();
ResourceLocalizationService rawService =
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
index 44bc1dbfe1..defd48955f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
@@ -104,21 +104,32 @@ public boolean isPmemCheckEnabled() {
conf.set(YarnConfiguration.NM_WEBAPP_ADDRESS, webAddr);
WebServer server = new WebServer(nmContext, resourceView,
new ApplicationACLsManager(conf), dirsHandler);
- server.init(conf);
- server.start();
- return server.getPort();
+ try {
+ server.init(conf);
+ server.start();
+ return server.getPort();
+ } finally {
+ server.stop();
+ healthChecker.stop();
+ }
}
@Test
public void testNMWebAppWithOutPort() throws IOException {
int port = startNMWebAppServer("0.0.0.0");
- Assert.assertTrue("Port is not updated", port > 0);
+ validatePortVal(port);
}
-
+
+ private void validatePortVal(int portVal) {
+ Assert.assertTrue("Port is not updated", portVal > 0);
+ Assert.assertTrue("Port is default "+ YarnConfiguration.DEFAULT_NM_PORT,
+ portVal !=YarnConfiguration.DEFAULT_NM_PORT);
+ }
+
@Test
public void testNMWebAppWithEphemeralPort() throws IOException {
int port = startNMWebAppServer("0.0.0.0:0");
- Assert.assertTrue("Port is not updated", port > 0);
+ validatePortVal(port);
}
@Test
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
index e023b3a98b..a4e57573f2 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/AdminService.java
@@ -92,8 +92,7 @@ public AdminService(Configuration conf, ResourceScheduler scheduler,
}
@Override
- public void init(Configuration conf) {
- super.init(conf);
+ public void serviceInit(Configuration conf) throws Exception {
masterServiceAddress = conf.getSocketAddr(
YarnConfiguration.RM_ADMIN_ADDRESS,
YarnConfiguration.DEFAULT_RM_ADMIN_ADDRESS,
@@ -101,9 +100,11 @@ public void init(Configuration conf) {
adminAcl = new AccessControlList(conf.get(
YarnConfiguration.YARN_ADMIN_ACL,
YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
+ super.serviceInit(conf);
}
- public void start() {
+ @Override
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
this.server =
@@ -122,15 +123,15 @@ public void start() {
this.server.start();
conf.updateConnectAddr(YarnConfiguration.RM_ADMIN_ADDRESS,
server.getListenerAddress());
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (this.server != null) {
this.server.stop();
}
- super.stop();
+ super.serviceStop();
}
private UserGroupInformation checkAcls(String method) throws YarnException {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
index d71c1de342..4a86c2e003 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
@@ -107,7 +107,7 @@ public ApplicationMasterService(RMContext rmContext, YarnScheduler scheduler) {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
@@ -133,7 +133,7 @@ public void start() {
this.bindAddress =
conf.updateConnectAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS,
server.getListenerAddress());
- super.start();
+ super.serviceStart();
}
@Private
@@ -447,10 +447,10 @@ public void refreshServiceAcls(Configuration configuration,
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (this.server != null) {
this.server.stop();
}
- super.stop();
+ super.serviceStop();
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
index 35c3a07334..88bc68b785 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
@@ -128,13 +128,13 @@ public ClientRMService(RMContext rmContext, YarnScheduler scheduler,
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
clientBindAddress = getBindAddress(conf);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
this.server =
@@ -157,16 +157,15 @@ public void start() {
// enable RM to short-circuit token operations directly to itself
RMDelegationTokenIdentifier.Renewer.setSecretManager(
rmDTSecretManager, clientBindAddress);
-
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (this.server != null) {
this.server.stop();
}
- super.stop();
+ super.serviceStop();
}
InetSocketAddress getBindAddress(Configuration conf) {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java
index 7f901c4952..c4af2d2a6c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NMLivelinessMonitor.java
@@ -37,12 +37,12 @@ public NMLivelinessMonitor(Dispatcher d) {
this.dispatcher = d.getEventHandler();
}
- public void init(Configuration conf) {
- super.init(conf);
+ public void serviceInit(Configuration conf) throws Exception {
int expireIntvl = conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
setExpireInterval(expireIntvl);
setMonitorInterval(expireIntvl/3);
+ super.serviceInit(conf);
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
index 1c502b09fb..234a39cf62 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/NodesListManager.java
@@ -57,7 +57,7 @@ public NodesListManager(RMContext rmContext) {
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
this.conf = conf;
@@ -83,7 +83,7 @@ public void init(Configuration conf) {
throw new YarnRuntimeException(ioe2);
}
}
- super.init(conf);
+ super.serviceInit(conf);
}
private void printConfiguredHosts() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
index 0efb174fec..ee0ca59354 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceManager.java
@@ -137,7 +137,7 @@ public RMContext getRMContext() {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
validateConfigs(conf);
@@ -257,7 +257,7 @@ public synchronized void init(Configuration conf) {
new RMNMInfo(this.rmContext, this.scheduler);
- super.init(conf);
+ super.serviceInit(conf);
}
@VisibleForTesting
@@ -398,17 +398,17 @@ public SchedulerEventDispatcher(ResourceScheduler scheduler) {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
this.shouldExitOnError =
conf.getBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY,
Dispatcher.DEFAULT_DISPATCHER_EXIT_ON_ERROR);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
this.eventProcessor.start();
- super.start();
+ super.serviceStart();
}
private final class EventProcessor implements Runnable {
@@ -448,7 +448,7 @@ public void run() {
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
this.stopped = true;
this.eventProcessor.interrupt();
try {
@@ -456,7 +456,7 @@ public synchronized void stop() {
} catch (InterruptedException e) {
throw new YarnRuntimeException(e);
}
- super.stop();
+ super.serviceStop();
}
@Override
@@ -577,7 +577,7 @@ protected void startWepApp() {
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
try {
doSecureLogin();
} catch(IOException ie) {
@@ -616,7 +616,7 @@ public void start() {
conf.set(YarnConfiguration.RM_WEBAPP_ADDRESS, resolvedAddress);
}
- super.start();
+ super.serviceStart();
/*synchronized(shutdown) {
try {
@@ -635,14 +635,20 @@ protected void doSecureLogin() throws IOException {
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (webApp != null) {
webApp.stop();
}
- rmDTSecretManager.stopThreads();
+ if (rmDTSecretManager != null) {
+ rmDTSecretManager.stopThreads();
+ }
- this.appTokenSecretManager.stop();
- this.containerTokenSecretManager.stop();
+ if (appTokenSecretManager != null) {
+ this.appTokenSecretManager.stop();
+ }
+ if (containerTokenSecretManager != null) {
+ this.containerTokenSecretManager.stop();
+ }
/*synchronized(shutdown) {
shutdown.set(true);
@@ -651,14 +657,16 @@ public void stop() {
DefaultMetricsSystem.shutdown();
- RMStateStore store = rmContext.getStateStore();
- try {
- store.close();
- } catch (Exception e) {
- LOG.error("Error closing store.", e);
+ if (rmContext != null) {
+ RMStateStore store = rmContext.getStateStore();
+ try {
+ store.close();
+ } catch (Exception e) {
+ LOG.error("Error closing store.", e);
+ }
}
-
- super.stop();
+
+ super.serviceStop();
}
protected ResourceTrackerService createResourceTrackerService() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
index 25776844b7..983480e1f7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ResourceTrackerService.java
@@ -99,7 +99,7 @@ public ResourceTrackerService(RMContext rmContext,
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
resourceTrackerAddress = conf.getSocketAddr(
YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
YarnConfiguration.DEFAULT_RM_RESOURCE_TRACKER_ADDRESS,
@@ -122,12 +122,12 @@ public synchronized void init(Configuration conf) {
YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES,
YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES);
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public synchronized void start() {
- super.start();
+ protected void serviceStart() throws Exception {
+ super.serviceStart();
// ResourceTrackerServer authenticates NodeManager via Kerberos if
// security is enabled, so no secretManager.
Configuration conf = getConfig();
@@ -151,11 +151,11 @@ public synchronized void start() {
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() throws Exception {
if (this.server != null) {
this.server.stop();
}
- super.stop();
+ super.serviceStop();
}
@SuppressWarnings("unchecked")
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
index a114d80e84..2b0ff7514b 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/ApplicationMasterLauncher.java
@@ -50,9 +50,10 @@ public ApplicationMasterLauncher(RMContext context) {
this.launcherHandlingThread = new LauncherThread();
}
- public void start() {
+ @Override
+ protected void serviceStart() throws Exception {
launcherHandlingThread.start();
- super.start();
+ super.serviceStart();
}
protected Runnable createRunnableLauncher(RMAppAttempt application,
@@ -69,7 +70,8 @@ private void launch(RMAppAttempt application) {
}
- public void stop() {
+ @Override
+ protected void serviceStop() throws Exception {
launcherHandlingThread.interrupt();
try {
launcherHandlingThread.join();
@@ -77,7 +79,6 @@ public void stop() {
LOG.info(launcherHandlingThread.getName() + " interrupted during join ",
ie); }
launcherPool.shutdown();
- super.stop();
}
private class LauncherThread extends Thread {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java
index 7e52d5a49d..2962085872 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/AMLivelinessMonitor.java
@@ -35,8 +35,8 @@ public AMLivelinessMonitor(Dispatcher d) {
this.dispatcher = d.getEventHandler();
}
- public void init(Configuration conf) {
- super.init(conf);
+ public void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf);
int expireIntvl = conf.getInt(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_AM_EXPIRY_INTERVAL_MS);
setExpireInterval(expireIntvl);
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java
index 74abe0a0cc..6d78728b57 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/ContainerAllocationExpirer.java
@@ -38,13 +38,13 @@ public ContainerAllocationExpirer(Dispatcher d) {
this.dispatcher = d.getEventHandler();
}
- public void init(Configuration conf) {
- super.init(conf);
+ public void serviceInit(Configuration conf) throws Exception {
int expireIntvl = conf.getInt(
YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS);
setExpireInterval(expireIntvl);
setMonitorInterval(expireIntvl/3);
+ super.serviceInit(conf);
}
@Override
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
index fbcbb70dcd..56d8f26382 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/security/DelegationTokenRenewer.java
@@ -88,19 +88,18 @@ public DelegationTokenRenewer() {
}
@Override
- public synchronized void init(Configuration conf) {
- super.init(conf);
+ protected synchronized void serviceInit(Configuration conf) throws Exception {
this.tokenKeepAliveEnabled =
conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED);
this.tokenRemovalDelayMs =
conf.getInt(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_NM_EXPIRY_INTERVAL_MS);
+ super.serviceInit(conf);
}
@Override
- public synchronized void start() {
- super.start();
+ protected void serviceStart() throws Exception {
dtCancelThread.start();
renewalTimer = new Timer(true);
@@ -110,10 +109,11 @@ public synchronized void start() {
"DelayedTokenCanceller");
delayedRemovalThread.start();
}
+ super.serviceStart();
}
@Override
- public synchronized void stop() {
+ protected void serviceStop() {
if (renewalTimer != null) {
renewalTimer.cancel();
}
@@ -133,8 +133,6 @@ public synchronized void stop() {
LOG.info("Interrupted while joining on delayed removal thread.", e);
}
}
-
- super.stop();
}
/**
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
index 0174499948..652b53c8fa 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
@@ -293,12 +293,12 @@ protected ClientRMService createClientRMService() {
return new ClientRMService(getRMContext(), getResourceScheduler(),
rmAppManager, applicationACLsManager, rmDTSecretManager) {
@Override
- public void start() {
+ protected void serviceStart() {
// override to not start rpc handler
}
@Override
- public void stop() {
+ protected void serviceStop() {
// don't do anything
}
};
@@ -313,12 +313,12 @@ protected ResourceTrackerService createResourceTrackerService() {
this.nmLivelinessMonitor, containerTokenSecretManager) {
@Override
- public void start() {
+ protected void serviceStart() {
// override to not start rpc handler
}
@Override
- public void stop() {
+ protected void serviceStop() {
// don't do anything
}
};
@@ -328,12 +328,12 @@ public void stop() {
protected ApplicationMasterService createApplicationMasterService() {
return new ApplicationMasterService(getRMContext(), scheduler) {
@Override
- public void start() {
+ protected void serviceStart() {
// override to not start rpc handler
}
@Override
- public void stop() {
+ protected void serviceStop() {
// don't do anything
}
};
@@ -343,7 +343,7 @@ public void stop() {
protected ApplicationMasterLauncher createAMLauncher() {
return new ApplicationMasterLauncher(getRMContext()) {
@Override
- public void start() {
+ protected void serviceStart() {
// override to not start rpc handler
}
@@ -353,7 +353,7 @@ public void handle(AMLauncherEvent appEvent) {
}
@Override
- public void stop() {
+ protected void serviceStop() {
// don't do anything
}
};
@@ -367,12 +367,12 @@ protected AdminService createAdminService(ClientRMService clientRMService,
this.nodesListManager, clientRMService, applicationMasterService,
resourceTrackerService) {
@Override
- public void start() {
+ protected void serviceStart() {
// override to not start rpc handler
}
@Override
- public void stop() {
+ protected void serviceStop() {
// don't do anything
}
};
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
index 0098893982..6bfe12cbe6 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMTokens.java
@@ -440,11 +440,11 @@ InetSocketAddress getBindAddress(Configuration conf) {
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if (rmDTSecretManager != null) {
rmDTSecretManager.stopThreads();
}
- super.stop();
+ super.serviceStop();
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
index cf4dc674ac..84944e4a74 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceManager.java
@@ -192,43 +192,43 @@ private void checkResourceUsage(
public void testResourceManagerInitConfigValidation() throws Exception {
Configuration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, -1);
+ resourceManager = new ResourceManager();
try {
resourceManager.init(conf);
fail("Exception is expected because the global max attempts" +
" is negative.");
} catch (YarnRuntimeException e) {
// Exception is expected.
- assertTrue("The thrown exception is not the expected one.",
- e.getMessage().startsWith(
- "Invalid global max attempts configuration"));
+ if (!e.getMessage().startsWith(
+ "Invalid global max attempts configuration")) throw e;
}
conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 2048);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_MB, 1024);
+ resourceManager = new ResourceManager();
try {
resourceManager.init(conf);
fail("Exception is expected because the min memory allocation is" +
" larger than the max memory allocation.");
} catch (YarnRuntimeException e) {
// Exception is expected.
- assertTrue("The thrown exception is not the expected one.",
- e.getMessage().startsWith(
- "Invalid resource scheduler memory"));
+ if (!e.getMessage().startsWith(
+ "Invalid resource scheduler memory")) throw e;
}
conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_VCORES, 2);
conf.setInt(YarnConfiguration.RM_SCHEDULER_MAXIMUM_ALLOCATION_VCORES, 1);
+ resourceManager = new ResourceManager();
try {
resourceManager.init(conf);
fail("Exception is expected because the min vcores allocation is" +
" larger than the max vcores allocation.");
} catch (YarnRuntimeException e) {
// Exception is expected.
- assertTrue("The thrown exception is not the expected one.",
- e.getMessage().startsWith(
- "Invalid resource scheduler vcores"));
+ if (!e.getMessage().startsWith(
+ "Invalid resource scheduler vcores")) throw e;
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
index 3bad5a3f55..b95cd68cd7 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/resourcetracker/TestNMExpiry.java
@@ -59,9 +59,9 @@ public TestNmLivelinessMonitor(Dispatcher dispatcher) {
}
@Override
- public void init(Configuration conf) {
+ public void serviceInit(Configuration conf) throws Exception {
conf.setLong(YarnConfiguration.RM_NM_EXPIRY_INTERVAL_MS, 1000);
- super.init(conf);
+ super.serviceInit(conf);
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java
index 883286d1f3..9f37ec4687 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java
@@ -128,7 +128,7 @@ public void ping() throws YarnException, IOException {
}
@Override
- public synchronized void start() {
+ protected void serviceStart() throws Exception {
Configuration conf = getConfig();
ClientToAMTokenSecretManager secretManager = null;
@@ -145,7 +145,7 @@ public synchronized void start() {
}
server.start();
this.address = NetUtils.getConnectAddress(server);
- super.start();
+ super.serviceStart();
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
index dc1d351949..5b30562094 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/MiniYARNCluster.java
@@ -92,7 +92,7 @@ public MiniYARNCluster(String testName, int noOfNodeManagers,
new Path(targetWorkDir.getAbsolutePath()), true);
} catch (Exception e) {
LOG.warn("COULD NOT CLEANUP", e);
- throw new YarnRuntimeException("could not cleanup test dir", e);
+ throw new YarnRuntimeException("could not cleanup test dir: "+ e, e);
}
if (Shell.WINDOWS) {
@@ -140,9 +140,10 @@ public MiniYARNCluster(String testName, int noOfNodeManagers,
}
@Override
- public void init(Configuration conf) {
- super.init(conf instanceof YarnConfiguration ? conf
- : new YarnConfiguration(conf));
+ public void serviceInit(Configuration conf) throws Exception {
+ super.serviceInit(conf instanceof YarnConfiguration ? conf
+ : new YarnConfiguration(
+ conf));
}
public File getTestWorkDir() {
@@ -172,23 +173,24 @@ public ResourceManagerWrapper() {
}
@Override
- public synchronized void start() {
+ public synchronized void serviceStart() throws Exception {
try {
getConfig().setBoolean(YarnConfiguration.IS_MINI_YARN_CLUSTER, true);
if (!getConfig().getBoolean(
YarnConfiguration.YARN_MINICLUSTER_FIXED_PORTS,
YarnConfiguration.DEFAULT_YARN_MINICLUSTER_FIXED_PORTS)) {
// pick free random ports.
+ String hostname = MiniYARNCluster.getHostname();
getConfig().set(YarnConfiguration.RM_ADDRESS,
- MiniYARNCluster.getHostname() + ":0");
+ hostname + ":0");
getConfig().set(YarnConfiguration.RM_ADMIN_ADDRESS,
- MiniYARNCluster.getHostname() + ":0");
+ hostname + ":0");
getConfig().set(YarnConfiguration.RM_SCHEDULER_ADDRESS,
- MiniYARNCluster.getHostname() + ":0");
+ hostname + ":0");
getConfig().set(YarnConfiguration.RM_RESOURCE_TRACKER_ADDRESS,
- MiniYARNCluster.getHostname() + ":0");
+ hostname + ":0");
getConfig().set(YarnConfiguration.RM_WEBAPP_ADDRESS,
- MiniYARNCluster.getHostname() + ":0");
+ hostname + ":0");
}
resourceManager = new ResourceManager() {
@Override
@@ -214,7 +216,7 @@ public void run() {
"ResourceManager failed to start. Final state is "
+ resourceManager.getServiceState());
}
- super.start();
+ super.serviceStart();
} catch (Throwable t) {
throw new YarnRuntimeException(t);
}
@@ -225,11 +227,11 @@ public void run() {
}
@Override
- public synchronized void stop() {
+ public synchronized void serviceStop() throws Exception {
if (resourceManager != null) {
resourceManager.stop();
}
- super.stop();
+ super.serviceStop();
if (Shell.WINDOWS) {
// On Windows, clean up the short temporary symlink that was created to
@@ -254,10 +256,10 @@ public NodeManagerWrapper(int i) {
index = i;
}
- public synchronized void init(Configuration conf) {
- Configuration config = new YarnConfiguration(conf);
- super.init(config);
- }
+ public synchronized void serviceInit(Configuration conf) throws Exception {
+ Configuration config = new YarnConfiguration(conf);
+ super.serviceInit(config);
+ }
/**
* Create local/log directories
@@ -279,7 +281,7 @@ private String prepareDirs(String dirType, int numDirs) {
return dirsString;
}
- public synchronized void start() {
+ public synchronized void serviceStart() throws Exception {
try {
// create nm-local-dirs and configure them for the nodemanager
String localDirsString = prepareDirs("local", numLocalDirs);
@@ -319,18 +321,18 @@ public void run() {
// RM could have failed.
throw new IOException("NodeManager " + index + " failed to start");
}
- super.start();
+ super.serviceStart();
} catch (Throwable t) {
throw new YarnRuntimeException(t);
}
}
@Override
- public synchronized void stop() {
+ public synchronized void serviceStop() throws Exception {
if (nodeManagers[index] != null) {
nodeManagers[index].stop();
}
- super.stop();
+ super.serviceStop();
}
}
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
index 4a9077b181..2be479fe2f 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxy.java
@@ -51,7 +51,7 @@ public WebAppProxy() {
}
@Override
- public void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
String auth = conf.get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION);
if (auth == null || "simple".equals(auth)) {
isSecurityEnabled = false;
@@ -81,11 +81,11 @@ public void init(Configuration conf) {
}
acl = new AccessControlList(conf.get(YarnConfiguration.YARN_ADMIN_ACL,
YarnConfiguration.DEFAULT_YARN_ADMIN_ACL));
- super.init(conf);
+ super.serviceInit(conf);
}
@Override
- public void start() {
+ protected void serviceStart() throws Exception {
try {
proxyServer = new HttpServer("proxy", bindAddress, port,
port == 0, getConfig(), acl);
@@ -99,11 +99,11 @@ public void start() {
LOG.fatal("Could not start proxy web server",e);
throw new YarnRuntimeException("Could not start proxy web server",e);
}
- super.start();
+ super.serviceStart();
}
@Override
- public void stop() {
+ protected void serviceStop() throws Exception {
if(proxyServer != null) {
try {
proxyServer.stop();
@@ -112,7 +112,7 @@ public void stop() {
throw new YarnRuntimeException("Error stopping proxy web server",e);
}
}
- super.stop();
+ super.serviceStop();
}
public void join() {
diff --git a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
index b0ba8a1771..0932b72e1c 100644
--- a/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
+++ b/hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-web-proxy/src/main/java/org/apache/hadoop/yarn/server/webproxy/WebAppProxyServer.java
@@ -51,7 +51,7 @@ public WebAppProxyServer() {
}
@Override
- public synchronized void init(Configuration conf) {
+ protected void serviceInit(Configuration conf) throws Exception {
Configuration config = new YarnConfiguration(conf);
try {
doSecureLogin(conf);
@@ -60,7 +60,7 @@ public synchronized void init(Configuration conf) {
}
proxy = new WebAppProxy();
addService(proxy);
- super.init(config);
+ super.serviceInit(config);
}
/**