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); } /**