YARN-530. Defined Service model strictly, implemented AbstractService for robust subclassing and migrated yarn-common services. Contributed by Steve Loughran.

YARN-117. Migrated rest of YARN to the new service model. Contributed by Steve Louhran.
MAPREDUCE-5298. Moved MapReduce services to YARN-530 stricter lifecycle. Contributed by Steve Loughran.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1492718 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-06-13 15:54:38 +00:00
parent 5244bf6860
commit 0928502029
107 changed files with 2785 additions and 1160 deletions

View File

@ -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

View File

@ -454,20 +454,32 @@
<Class name="~org\.apache\.hadoop\.mapreduce\.v2\.proto.*" />
</Match>
<Match>
<Class name="org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator" />
<Field name="mapResourceReqt" />
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<Match>
<Class name="org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator" />
<Field name="reduceResourceReqt" />
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<!--
The above 2 fields are accessed locally and only via methods that are synchronized.
The below fields are accessed locally and only via methods that are synchronized.
-->
<Match>
<Class name="org.apache.hadoop.mapreduce.v2.app.rm.RMContainerAllocator" />
<Or>
<Field name="mapResourceReqt" />
<Field name="reduceResourceReqt" />
<Field name="maxReduceRampupLimit" />
<Field name="reduceSlowStart" />
</Or>
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<Match>
<Class name="org.apache.hadoop.mapreduce.v2.app.local.LocalContainerAllocator" />
<Or>
<Field name="retryInterval" />
<Field name="retrystartTime" />
</Or>
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<Match>
<Class name="org.apache.hadoop.mapreduce.v2.app.commit.CommitterEventHandler" />
<Field name="commitThreadCancelTimeoutMs" />
<Bug pattern="IS2_INCONSISTENT_SYNC" />
</Match>
<Match>
<Class name="org.apache.hadoop.mapred.ShuffleHandler" />

View File

@ -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 class LocalContainerLauncher extends AbstractService implements
// 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

View File

@ -95,17 +95,17 @@ public class TaskAttemptListenerImpl extends CompositeService
}
@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 @@ public class TaskAttemptListenerImpl extends CompositeService
}
@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

View File

@ -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 class JobHistoryCopyService extends CompositeService implements HistoryEv
}
@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 class JobHistoryCopyService extends CompositeService implements HistoryEv
}
@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 {

View File

@ -115,7 +115,7 @@ public class JobHistoryEventHandler extends AbstractService
* 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 class JobHistoryEventHandler extends AbstractService
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 @@ public class JobHistoryEventHandler extends AbstractService
}
@Override
public void start() {
protected void serviceStart() throws Exception {
eventHandlingThread = new Thread(new Runnable() {
@Override
public void run() {
@ -275,38 +275,48 @@ public class JobHistoryEventHandler extends AbstractService
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 class JobHistoryEventHandler extends AbstractService
}
}
LOG.info("Stopped JobHistoryEventHandler. super.stop()");
super.stop();
super.serviceStop();
}
protected EventWriter createEventWriter(Path historyFilePath)
@ -725,6 +735,7 @@ public class JobHistoryEventHandler extends AbstractService
@Override
public void run() {
LOG.debug("In flush timer task");
synchronized (lock) {
try {
if (!metaInfo.isTimerShutDown() && shouldRun)
@ -790,7 +801,14 @@ public class JobHistoryEventHandler extends AbstractService
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 @@ public class JobHistoryEventHandler extends AbstractService
}
void writeEvent(HistoryEvent event) throws IOException {
LOG.debug("Writing event");
synchronized (lock) {
if (writer != null) {
writer.write(event);
@ -849,6 +868,9 @@ public class JobHistoryEventHandler extends AbstractService
}
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 @@ public class JobHistoryEventHandler extends AbstractService
}
void shutDownTimer() throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Shutting down timer "+ toString());
}
synchronized (lock) {
isTimerShutDown = true;
flushTimer.cancel();

View File

@ -128,6 +128,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
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 class MRAppMaster extends CompositeService {
}
@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 class MRAppMaster extends CompositeService {
addIfService(historyService);
}
super.init(conf);
super.serviceInit(conf);
} // end of init()
protected Dispatcher createDispatcher() {
@ -784,7 +785,7 @@ public class MRAppMaster extends CompositeService {
}
@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 class MRAppMaster extends CompositeService {
}
((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 @@ public class MRAppMaster extends CompositeService {
}
@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 class MRAppMaster extends CompositeService {
}
((Service)this.containerLauncher).init(getConfig());
((Service)this.containerLauncher).start();
super.start();
super.serviceStart();
}
@Override
@ -861,9 +862,9 @@ public class MRAppMaster extends CompositeService {
}
@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 @@ public class MRAppMaster extends CompositeService {
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
try {
if(isLastAMRetry) {
cleanupStagingDir();
@ -884,7 +885,7 @@ public class MRAppMaster extends CompositeService {
} catch (IOException io) {
LOG.error("Failed to cleanup staging dir: ", io);
}
super.stop();
super.serviceStop();
}
}
@ -951,7 +952,7 @@ public class MRAppMaster extends CompositeService {
@SuppressWarnings("unchecked")
@Override
public void start() {
protected void serviceStart() throws Exception {
amInfos = new LinkedList<AMInfo>();
completedTasksFromPreviousRun = new HashMap<TaskId, TaskInfo>();
@ -1011,7 +1012,7 @@ public class MRAppMaster extends CompositeService {
}
//start all the components
super.start();
super.serviceStart();
// All components have started, start the job.
startJobs();

View File

@ -85,26 +85,28 @@ public class TaskHeartbeatHandler extends AbstractService {
}
@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) {

View File

@ -112,7 +112,7 @@ public class MRClientService extends AbstractService
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 class MRClientService extends AbstractService
} 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 @@ public class MRClientService extends AbstractService
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

View File

@ -87,8 +87,8 @@ public class CommitterEventHandler extends AbstractService
}
@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 class CommitterEventHandler extends AbstractService
}
@Override
public void start() {
protected void serviceStart() throws Exception {
ThreadFactory tf = new ThreadFactoryBuilder()
.setNameFormat("CommitterEvent Processor #%d")
.build();
@ -134,7 +134,7 @@ public class CommitterEventHandler extends AbstractService
});
eventHandlingThread.setName("CommitterEvent Handler");
eventHandlingThread.start();
super.start();
super.serviceStart();
}
@ -148,14 +148,18 @@ public class CommitterEventHandler extends AbstractService
}
@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 {

View File

@ -239,7 +239,7 @@ public class ContainerLauncherImpl extends AbstractService implements
}
@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 class ContainerLauncherImpl extends AbstractService implements
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 class ContainerLauncherImpl extends AbstractService implements
};
eventHandlingThread.setName("ContainerLauncher Event Handler");
eventHandlingThread.start();
super.start();
super.serviceStart();
}
private void shutdownAllContainers() {
@ -328,16 +328,20 @@ public class ContainerLauncherImpl extends AbstractService implements
}
}
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) {

View File

@ -80,8 +80,8 @@ public class LocalContainerAllocator extends RMCommunicator
}
@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);

View File

@ -99,22 +99,22 @@ public abstract class RMCommunicator extends AbstractService
}
@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 @@ public abstract class RMCommunicator extends AbstractService
}
@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() {

View File

@ -156,8 +156,8 @@ public class RMContainerAllocator extends RMContainerRequestor
}
@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 class RMContainerAllocator extends RMContainerRequestor
}
@Override
public void start() {
protected void serviceStart() throws Exception {
this.eventHandlingThread = new Thread() {
@SuppressWarnings("unchecked")
@Override
@ -208,7 +208,7 @@ public class RMContainerAllocator extends RMContainerRequestor
}
};
this.eventHandlingThread.start();
super.start();
super.serviceStart();
}
@Override
@ -242,13 +242,15 @@ public class RMContainerAllocator extends RMContainerRequestor
}
@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: ");
}

View File

@ -124,8 +124,8 @@ public abstract class RMContainerRequestor extends RMCommunicator {
}
@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);

View File

@ -166,7 +166,7 @@ public class DefaultSpeculator extends AbstractService implements
// looking for speculation opportunities
@Override
public void start() {
protected void serviceStart() throws Exception {
Runnable speculationBackgroundCore
= new Runnable() {
@Override
@ -202,17 +202,17 @@ public class DefaultSpeculator extends AbstractService implements
(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

View File

@ -350,7 +350,7 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler {
}
@Override
public void start() {
protected void serviceStart() {
}
@Override

View File

@ -207,7 +207,7 @@ public class MRApp extends MRAppMaster {
}
@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 class MRApp extends MRAppMaster {
throw new YarnRuntimeException("Error creating staging dir", e);
}
super.init(conf);
super.serviceInit(conf);
if (this.clusterInfo != null) {
getContext().getClusterInfo().setMinContainerCapability(
this.clusterInfo.getMinContainerCapability());

View File

@ -130,7 +130,7 @@ public class MRAppBenchmark {
}
}
@Override
public void start() {
protected void serviceStart() throws Exception {
thread = new Thread(new Runnable() {
@Override
public void run() {
@ -168,13 +168,15 @@ public class MRAppBenchmark {
}
});
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();
}
}
}

View File

@ -261,10 +261,11 @@ public class TestFail {
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);
}
};
}

View File

@ -438,9 +438,9 @@ class MRAppMasterTest extends MRAppMaster {
}
@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 @@ class MRAppMasterTest extends MRAppMaster {
}
@Override
public void start() {
protected void serviceStart() throws Exception {
if (overrideStart) {
try {
UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
@ -483,7 +483,7 @@ class MRAppMasterTest extends MRAppMaster {
fail(e.getMessage());
}
} else {
super.start();
super.serviceStart();
}
}

View File

@ -56,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
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 @@ import org.junit.Test;
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 @@ import org.junit.Test;
}
@Override
public void start() {
super.start();
public void serviceStart() throws Exception {
super.serviceStart();
DefaultMetricsSystem.shutdown();
}
@ -329,9 +333,9 @@ import org.junit.Test;
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
stoppedContainerAllocator = true;
super.stop();
super.serviceStop();
}
}

View File

@ -60,7 +60,8 @@ public class CachedHistoryStorage extends AbstractService implements
@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 class CachedHistoryStorage extends AbstractService implements
return super.size() > loadedJobCacheSize;
}
});
super.init(conf);
}
public CachedHistoryStorage() {

View File

@ -111,7 +111,7 @@ public class HistoryClientService extends AbstractService {
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 class HistoryClientService extends AbstractService {
server.getListenerAddress());
LOG.info("Instantiated MRClientService at " + this.bindAddress);
super.start();
super.serviceStart();
}
private void initializeWebApp(Configuration conf) {
@ -155,14 +155,14 @@ public class HistoryClientService extends AbstractService {
}
@Override
public void stop() {
protected void serviceStop() throws Exception {
if (server != null) {
server.stop();
}
if (webApp != null) {
webApp.stop();
}
super.stop();
super.serviceStop();
}
@Private

View File

@ -244,6 +244,9 @@ public class HistoryFileManager extends AbstractService {
}
public void delete(HistoryFileInfo fileInfo) {
if (LOG.isDebugEnabled()) {
LOG.debug("Removing from cache " + fileInfo);
}
cache.remove(fileInfo.getJobId());
}
@ -275,6 +278,10 @@ public class HistoryFileManager extends AbstractService {
} 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 class HistoryFileManager extends AbstractService {
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 @@ public class HistoryFileManager extends AbstractService {
}
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 class HistoryFileManager extends AbstractService {
}
@Override
public void init(Configuration conf) {
protected void serviceInit(Configuration conf) throws Exception {
this.conf = conf;
int serialNumberLowDigits = 3;
@ -519,7 +541,7 @@ public class HistoryFileManager extends AbstractService {
moveToDoneExecutor = new ThreadPoolExecutor(numMoveThreads, numMoveThreads,
1, TimeUnit.HOURS, new LinkedBlockingQueue<Runnable>(), tf);
super.init(conf);
super.serviceInit(conf);
}
private void mkdir(FileContext fc, Path path, FsPermission fsp)
@ -665,7 +687,7 @@ public class HistoryFileManager extends AbstractService {
// case where we are looking for a particular job.
List<FileStatus> 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 @@ public class HistoryFileManager extends AbstractService {
* @throws IOException
*/
private void scanIntermediateDirectory(final Path absPath) throws IOException {
if (LOG.isDebugEnabled()) {
LOG.debug("Scanning intermediate dir " + absPath);
}
List<FileStatus> 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 @@ public class HistoryFileManager extends AbstractService {
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 class HistoryFileManager extends AbstractService {
}
} else if (old != null && !old.isMovePending()) {
//This is a duplicate so just delete it
if (LOG.isDebugEnabled()) {
LOG.debug("Duplicate: deleting");
}
fileInfo.delete();
}
}

View File

@ -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 class JobHistory extends AbstractService implements HistoryContext {
}
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 class JobHistory extends AbstractService implements HistoryContext {
.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 class JobHistory extends AbstractService implements HistoryContext {
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() {

View File

@ -65,7 +65,7 @@ public class JobHistoryServer extends CompositeService {
}
@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 class JobHistoryServer extends CompositeService {
addService(jobHistoryService);
addService(clientService);
addService(aggLogDelService);
super.init(config);
super.serviceInit(config);
}
protected JHSDelegationTokenSecretManager createJHSSecretManager(
@ -109,23 +109,25 @@ public class JobHistoryServer extends CompositeService {
}
@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

View File

@ -206,7 +206,7 @@ public class TestJobHistoryEvents {
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();

View File

@ -76,12 +76,15 @@ public class TestJobHistoryServer {
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 class TestJobHistoryServer {
@After
public void stop(){
if(historyServer !=null && !STATE.STOPPED.equals(historyServer.getServiceState())){
if(historyServer != null) {
historyServer.stop();
}
}

View File

@ -233,7 +233,7 @@ public class TestClientRedirect {
}
@Override
public void init(Configuration conf) {
protected void serviceInit(Configuration conf) throws Exception {
clientServiceBindAddress = RMADDRESS;
/*
clientServiceBindAddress = conf.get(
@ -241,11 +241,11 @@ public class TestClientRedirect {
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 class TestClientRedirect {
this.server = rpc.getServer(ClientRMProtocol.class, this,
clientBindAddress, clientServerConf, null, 1);
this.server.start();
super.start();
super.serviceStart();
}
@Override
@ -416,9 +416,12 @@ public class TestClientRedirect {
amRunning = true;
}
public void stop() {
server.stop();
super.stop();
@Override
protected void serviceStop() throws Exception {
if (server != null) {
server.stop();
}
super.serviceStop();
amRunning = false;
}

View File

@ -67,7 +67,7 @@ public class TestResourceMgrDelegate {
ResourceMgrDelegate delegate = new ResourceMgrDelegate(
new YarnConfiguration()) {
@Override
public synchronized void start() {
protected void serviceStart() {
this.rmClient = applicationsManager;
}
};
@ -110,7 +110,7 @@ public class TestResourceMgrDelegate {
ResourceMgrDelegate resourceMgrDelegate = new ResourceMgrDelegate(
new YarnConfiguration()) {
@Override
public synchronized void start() {
protected void serviceStart() {
this.rmClient = applicationsManager;
}
};

View File

@ -200,7 +200,7 @@ public class TestYARNRunner extends TestCase {
final ClientRMProtocol clientRMProtocol = mock(ClientRMProtocol.class);
ResourceMgrDelegate delegate = new ResourceMgrDelegate(conf) {
@Override
public synchronized void start() {
protected void serviceStart() {
this.rmClient = clientRMProtocol;
}
};

View File

@ -110,7 +110,7 @@ public class TestYarnClientProtocolProvider extends TestCase {
ResourceMgrDelegate rmgrDelegate = new ResourceMgrDelegate(
new YarnConfiguration(conf)) {
@Override
public synchronized void start() {
protected void serviceStart() {
this.rmClient = cRMProtocol;
}
};

View File

@ -67,7 +67,7 @@ public class MiniMRYarnCluster extends MiniYARNCluster {
}
@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 class MiniMRYarnCluster extends MiniYARNCluster {
// 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 class MiniMRYarnCluster extends MiniYARNCluster {
}
@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 class MiniMRYarnCluster extends MiniYARNCluster {
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 class MiniMRYarnCluster extends MiniYARNCluster {
}
@Override
public synchronized void stop() {
public synchronized void serviceStop() throws Exception {
if (historyServer != null) {
historyServer.stop();
}
super.stop();
super.serviceStop();
}
}

View File

@ -267,7 +267,7 @@ public class ShuffleHandler extends AbstractService
}
@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 class ShuffleHandler extends AbstractService
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 class ShuffleHandler extends AbstractService
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

View File

@ -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.

View File

@ -24,6 +24,11 @@
<Class name="~org\.apache\.hadoop\.yarn\.ipc\.RpcProtos.*" />
</Match>
<Match>
<Class name="org.apache.hadoop.yarn.service.AbstractService" />
<Bug pattern="JLM_JSR166_UTILCONCURRENT_MONITORENTER" />
</Match>
<!-- Ignore unchecked Event casts -->
<Match>
<Class name="org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl" />
@ -172,6 +177,11 @@
<Field name="renewalTimer" />
<Bug code="IS"/>
</Match>
<Match>
<Class name="org.apache.hadoop.yarn.client.NMClientImpl$NMCommunicator"/>
<Field name="containerManager" />
<Bug pattern="IS2_INCONSISTENT_SYNC"/>
</Match>
<!-- Don't care if putIfAbsent value is ignored -->
<Match>

View File

@ -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 class TestDistributedShell {
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 class TestDistributedShell {
@AfterClass
public static void tearDown() throws IOException {
if (yarnCluster != null) {
yarnCluster.stop();
yarnCluster = null;
try {
yarnCluster.stop();
} finally {
yarnCluster = null;
}
}
}

View File

@ -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 class TestUnmanagedAMLauncher {
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 class TestUnmanagedAMLauncher {
@AfterClass
public static void tearDown() throws IOException {
if (yarnCluster != null) {
yarnCluster.stop();
yarnCluster = null;
try {
yarnCluster.stop();
} finally {
yarnCluster = null;
}
}
}

View File

@ -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

View File

@ -132,16 +132,16 @@ public class AMRMClientAsync<T extends ContainerRequest> extends AbstractService
}
@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 class AMRMClientAsync<T extends ContainerRequest> extends AbstractService
* 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 class AMRMClientAsync<T extends ContainerRequest> extends AbstractService
} catch (InterruptedException ex) {
LOG.error("Error joining with hander thread", ex);
}
super.stop();
super.serviceStop();
}
public void setHeartbeatInterval(int interval) {

View File

@ -151,12 +151,12 @@ public class AMRMClientImpl<T extends ContainerRequest>
}
@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 class AMRMClientImpl<T extends ContainerRequest>
}
});
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

View File

@ -166,18 +166,18 @@ public class NMClientAsync extends AbstractService {
}
@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 class NMClientAsync extends AbstractService {
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(

View File

@ -86,7 +86,7 @@ public class NMClientImpl extends AbstractService implements NMClient {
new ConcurrentHashMap<ContainerId, StartedContainer>();
//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 class NMClientImpl extends AbstractService implements NMClient {
}
@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 class NMClientImpl extends AbstractService implements NMClient {
}
@Override
public synchronized void start() {
protected void serviceStart() throws Exception {
final YarnRPC rpc = YarnRPC.create(getConfig());
final InetSocketAddress containerAddress =
@ -195,10 +195,11 @@ public class NMClientImpl extends AbstractService implements NMClient {
});
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 class NMClientImpl extends AbstractService implements NMClient {
containerAddress);
}
}
super.serviceStop();
}
public synchronized Map<String, ByteBuffer> startContainer(

View File

@ -79,7 +79,11 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
}
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 @@ public class YarnClientImpl extends AbstractService implements YarnClient {
}
@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 class YarnClientImpl extends AbstractService implements YarnClient {
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

View File

@ -153,24 +153,37 @@ public class TestNMClient {
@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 class 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<Container> allocateContainers(
@ -250,9 +268,12 @@ public class TestNMClient {
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 @@ public class TestNMClient {
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 @@ public class TestNMClient {
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

View File

@ -52,6 +52,8 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
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();

View File

@ -82,24 +82,24 @@ public class AsyncDispatcher extends AbstractService implements Dispatcher {
}
@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 class AsyncDispatcher extends AbstractService implements Dispatcher {
}
// stop all the components
super.stop();
super.serviceStop();
}
@SuppressWarnings("unchecked")

View File

@ -126,7 +126,8 @@ public class AggregatedLogDeletionService extends AbstractService {
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 class AggregatedLogDeletionService extends AbstractService {
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();
}
}

View File

@ -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,14 +53,51 @@ 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<ServiceStateChangeListener> listeners =
new ArrayList<ServiceStateChangeListener>();
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<LifecycleEvent> lifecycleHistory
= new ArrayList<LifecycleEvent>(5);
/**
* Map of blocking dependencies
*/
private final Map<String,String> blockerMap = new HashMap<String, String>();
private final Object stateChangeLock = new Object();
/**
* Construct the service.
@ -61,68 +105,289 @@ public abstract class AbstractService implements Service {
*/
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 abstract class AbstractService implements Service {
}
/**
* 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<LifecycleEvent> getLifecycleHistory() {
return new ArrayList<LifecycleEvent>(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<String, String> getBlockers() {
synchronized (blockerMap) {
Map<String, String> map = new HashMap<String, String>(blockerMap);
return map;
}
}
}

View File

@ -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<Service> serviceList = new ArrayList<Service>();
/**
* 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<Service> serviceList = new ArrayList<Service>();
public CompositeService(String name) {
super(name);
}
public Collection<Service> 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<Service> 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<Service> 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<Service> 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 -<i>after all services are stopped</i>
*/
private synchronized void stop(int numOfServicesStarted,
boolean stopOnlyStartedServices) {
// stop in reverse order of start
Exception firstException = null;
List<Service> 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,12 +158,7 @@ public class CompositeService extends AbstractService {
@Override
public void run() {
try {
// Stop the Composite Service
compositeService.stop();
} catch (Throwable t) {
LOG.info("Error stopping " + compositeService.getName(), t);
}
ServiceOperations.stopQuietly(compositeService);
}
}

View File

@ -20,6 +20,10 @@ package org.apache.hadoop.yarn.service;
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 class FilterService implements Service {
service.stop();
}
@Override
public void close() throws IOException {
service.close();
}
@Override
public void register(ServiceStateChangeListener listener) {
service.register(listener);
@ -73,4 +82,34 @@ public class FilterService implements Service {
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<LifecycleEvent> getLifecycleHistory() {
return service.getLifecycleHistory();
}
@Override
public Map<String, String> getBlockers() {
return service.getBlockers();
}
}

View File

@ -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;
}

View File

@ -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());
}
}

View File

@ -20,34 +20,77 @@ package org.apache.hadoop.yarn.service;
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 interface Service {
/**
* 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 interface Service {
* 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<LifecycleEvent> getLifecycleHistory();
/**
* Get the blockers on a service -remote dependencies
* that are stopping the service from being <i>live</i>.
* @return a (snapshotted) map of blocker name-&gt;description values
*/
public Map<String, String> getBlockers();
}

View File

@ -21,6 +21,11 @@ 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.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.
* <p/>
* The service state is checked <i>before</i> the operation begins.
* This process is <i>not</i> 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.
* <p/>
* The service state is checked <i>before</i> the operation begins.
* This process is <i>not</i> 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.
* <p/>
* The service state is checked <i>before</i> the operation begins.
* This process is <i>not</i> 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.
* <p/>Do nothing if the service is null or not
@ -111,10 +48,7 @@ public final class ServiceOperations {
*/
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 final class ServiceOperations {
* @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<ServiceStateChangeListener> listeners =
new ArrayList<ServiceStateChangeListener>();
/**
* 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);
}
}
}
}

View File

@ -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);
}
}
}

View File

@ -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();
}
}

View File

@ -49,21 +49,21 @@ public abstract class AbstractLivelinessMonitor<O> extends AbstractService {
}
@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);

View File

@ -55,13 +55,7 @@ public class BreakableService extends AbstractService {
}
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 class BreakableService extends AbstractService {
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 class BreakableService extends AbstractService {
* 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();
}
}
}

View File

@ -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<Service.STATE> stateEventList = new ArrayList<Service.STATE>(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<Service.STATE> 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 + "]";
}
}

View File

@ -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);
}
}

View File

@ -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 class TestServiceLifecycle extends ServiceAssert {
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 class TestServiceLifecycle extends ServiceAssert {
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 class TestServiceLifecycle extends ServiceAssert {
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 class TestServiceLifecycle extends ServiceAssert {
//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 class TestServiceLifecycle extends ServiceAssert {
//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 class TestServiceLifecycle extends ServiceAssert {
//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 <code>AbstractService</code> implementation copies the list
* to an array in a <code>synchronized</code> 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();
}
}
}

View File

@ -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);
}
}

View File

@ -21,10 +21,15 @@ package org.apache.hadoop.yarn.util;
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 class TestCompositeService {
// 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 class TestCompositeService {
}
// 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 class TestCompositeService {
+ " 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 class TestCompositeService {
}
}
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 class TestCompositeService {
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 class TestCompositeService {
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 class TestCompositeService {
}
@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() {

View File

@ -30,7 +30,6 @@ import org.apache.hadoop.conf.Configuration;
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 class DeletionService extends AbstractService {
}
@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 class DeletionService extends AbstractService {
}
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();
}
/**

View File

@ -113,7 +113,7 @@ public class LocalDirsHandlerService extends AbstractService {
*
*/
@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 class LocalDirsHandlerService extends AbstractService {
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 class LocalDirsHandlerService extends AbstractService {
* 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();
}
/**

View File

@ -39,13 +39,13 @@ public class NodeHealthCheckerService extends CompositeService {
}
@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);
}
/**

View File

@ -197,7 +197,7 @@ public class NodeHealthScriptRunner extends AbstractService {
* 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 class NodeHealthScriptRunner extends AbstractService {
String[] args = conf.getStrings(YarnConfiguration.NM_HEALTH_CHECK_SCRIPT_OPTS,
new String[] {});
timer = new NodeHealthMonitorExecutor(args);
super.serviceInit(conf);
}
/**
@ -216,7 +217,7 @@ public class NodeHealthScriptRunner extends AbstractService {
*
*/
@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 class NodeHealthScriptRunner extends AbstractService {
// Start the timer task immediately and
// then periodically at interval time.
nodeHealthScriptScheduler.scheduleAtFixedRate(timer, 0, intervalTime);
super.serviceStart();
}
/**
@ -233,11 +235,13 @@ public class NodeHealthScriptRunner extends AbstractService {
*
*/
@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) {

View File

@ -128,7 +128,7 @@ public class NodeManager extends CompositeService
}
@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 class NodeManager extends CompositeService
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() {

View File

@ -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 class NodeStatusUpdaterImpl extends AbstractService implements
}
@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 class NodeStatusUpdaterImpl extends AbstractService implements
" 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 class NodeStatusUpdaterImpl extends AbstractService implements
// 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 class NodeStatusUpdaterImpl extends AbstractService implements
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
// Interrupt the updater.
this.isStopped = true;
super.stop();
super.serviceStop();
}
protected void rebootNodeStatusUpdater() {

View File

@ -30,7 +30,6 @@ import org.apache.commons.logging.LogFactory;
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 class AuxServices extends AbstractService
}
@Override
public void init(Configuration conf) {
public void serviceInit(Configuration conf) throws Exception {
Collection<String> auxNames = conf.getStringCollection(
YarnConfiguration.NM_AUX_SERVICES);
for (final String sName : auxNames) {
@ -110,11 +109,11 @@ public class AuxServices extends AbstractService
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<String, AuxiliaryService> entry : serviceMap.entrySet()) {
@ -127,11 +126,11 @@ public class AuxServices extends AbstractService
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 class AuxServices extends AbstractService
serviceMeta.clear();
}
} finally {
super.stop();
super.serviceStop();
}
}

View File

@ -177,13 +177,13 @@ public class ContainerManagerImpl extends CompositeService implements
}
@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 @@ public class ContainerManagerImpl extends CompositeService implements
}
@Override
public void start() {
protected void serviceStart() throws Exception {
// Enqueue user dirs in deletion context
@ -254,7 +254,7 @@ public class ContainerManagerImpl extends CompositeService implements
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 @@ public class ContainerManagerImpl extends CompositeService implements
}
@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.

View File

@ -91,20 +91,20 @@ public class ContainersLauncher extends AbstractService
}
@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

View File

@ -194,7 +194,7 @@ public class ResourceLocalizationService extends CompositeService
}
@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 class ResourceLocalizationService extends CompositeService
localizerTracker = createLocalizerTracker(conf);
addService(localizerTracker);
dispatcher.register(LocalizerEventType.class, localizerTracker);
super.init(conf);
super.serviceInit(conf);
}
@Override
@ -248,7 +248,7 @@ public class ResourceLocalizationService extends CompositeService
}
@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 class ResourceLocalizationService extends CompositeService
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 @@ public class ResourceLocalizationService extends CompositeService
}
@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 @@ public class ResourceLocalizationService extends CompositeService
}
@Override
public synchronized void start() {
public synchronized void serviceStart() throws Exception {
publicLocalizer.start();
super.start();
super.serviceStart();
}
public LocalizerHeartbeatResponse processHeartbeat(LocalizerStatus status) {
@ -559,12 +559,12 @@ public class ResourceLocalizationService extends CompositeService
}
@Override
public void stop() {
public void serviceStop() throws Exception {
for (LocalizerRunner localizer : privLocalizers.values()) {
localizer.interrupt();
}
publicLocalizer.interrupt();
super.stop();
super.serviceStop();
}
@Override

View File

@ -114,7 +114,7 @@ public class LogAggregationService extends AbstractService implements
.build());
}
public synchronized void init(Configuration conf) {
protected void serviceInit(Configuration conf) throws Exception {
this.remoteRootLogDir =
new Path(conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR));
@ -122,22 +122,22 @@ public class LogAggregationService extends AbstractService implements
conf.get(YarnConfiguration.NM_REMOTE_APP_LOG_DIR_SUFFIX,
YarnConfiguration.DEFAULT_NM_REMOTE_APP_LOG_DIR_SUFFIX);
super.init(conf);
super.serviceInit(conf);
}
@Override
public synchronized void start() {
protected void serviceStart() throws Exception {
// NodeId is only available during start, the following cannot be moved
// anywhere else.
this.nodeId = this.context.getNodeId();
super.start();
super.serviceStart();
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
LOG.info(this.getName() + " waiting for pending aggregation during exit");
stopAggregators();
super.stop();
super.serviceStop();
}
private void stopAggregators() {

View File

@ -69,16 +69,16 @@ public class NonAggregatingLogHandler extends AbstractService implements
}
@Override
public void init(Configuration conf) {
protected void serviceInit(Configuration conf) throws Exception {
// Default 3 hours.
this.deleteDelaySeconds =
conf.getLong(YarnConfiguration.NM_LOG_RETAIN_SECONDS, 3 * 60 * 60);
sched = createScheduledThreadPoolExecutor(conf);
super.init(conf);
super.serviceInit(conf);
}
@Override
public void stop() {
protected void serviceStop() throws Exception {
if (sched != null) {
sched.shutdown();
boolean isShutdown = false;
@ -92,7 +92,7 @@ public class NonAggregatingLogHandler extends AbstractService implements
sched.shutdownNow();
}
}
super.stop();
super.serviceStop();
}
@SuppressWarnings("unchecked")

View File

@ -85,7 +85,7 @@ public class ContainersMonitorImpl extends AbstractService implements
}
@Override
public synchronized void init(Configuration conf) {
protected void serviceInit(Configuration conf) throws Exception {
this.monitoringInterval =
conf.getLong(YarnConfiguration.NM_CONTAINER_MON_INTERVAL_MS,
YarnConfiguration.DEFAULT_NM_CONTAINER_MON_INTERVAL_MS);
@ -151,7 +151,7 @@ public class ContainersMonitorImpl extends AbstractService implements
1) + "). Thrashing might happen.");
}
}
super.init(conf);
super.serviceInit(conf);
}
private boolean isEnabled() {
@ -175,15 +175,15 @@ public class ContainersMonitorImpl extends AbstractService implements
}
@Override
public synchronized void start() {
protected void serviceStart() throws Exception {
if (this.isEnabled()) {
this.monitoringThread.start();
}
super.start();
super.serviceStart();
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
if (this.isEnabled()) {
this.monitoringThread.interrupt();
try {
@ -192,7 +192,7 @@ public class ContainersMonitorImpl extends AbstractService implements
;
}
}
super.stop();
super.serviceStop();
}
private static class ProcessTreeInfo {

View File

@ -53,12 +53,7 @@ public class WebServer extends AbstractService {
}
@Override
public synchronized void init(Configuration conf) {
super.init(conf);
}
@Override
public synchronized void start() {
protected void serviceStart() throws Exception {
String bindAddress = getConfig().get(YarnConfiguration.NM_WEBAPP_ADDRESS,
YarnConfiguration.DEFAULT_NM_WEBAPP_ADDRESS);
LOG.info("Instantiating NMWebApp at " + bindAddress);
@ -70,9 +65,9 @@ public class WebServer extends AbstractService {
} catch (Exception e) {
String msg = "NMWebapps failed to start.";
LOG.error(msg, e);
throw new YarnRuntimeException(msg);
throw new YarnRuntimeException(msg, e);
}
super.start();
super.serviceStart();
}
public int getPort() {
@ -80,11 +75,12 @@ public class WebServer extends AbstractService {
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
if (this.webApp != null) {
LOG.debug("Stopping webapp");
this.webApp.stop();
}
super.stop();
super.serviceStop();
}
public static class NMWebApp extends WebApp implements YarnWebParams {

View File

@ -138,9 +138,9 @@ public class TestDeletionService {
}
DeletionService del =
new DeletionService(new FakeDefaultContainerExecutor());
del.init(new Configuration());
del.start();
try {
del.init(new Configuration());
del.start();
for (Path p : content) {
assertTrue(lfs.util().exists(new Path(baseDirs.get(0), p)));
del.delete((Long.parseLong(p.getName()) % 2) == 0 ? null : "dingo",
@ -176,9 +176,9 @@ public class TestDeletionService {
conf.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, -1);
exec.setConf(conf);
DeletionService del = new DeletionService(exec);
del.init(conf);
del.start();
try {
del.init(conf);
del.start();
for (Path p : dirs) {
del.delete((Long.parseLong(p.getName()) % 2) == 0 ? null : "dingo", p,
null);
@ -201,9 +201,9 @@ public class TestDeletionService {
DeletionService del = new DeletionService(Mockito.mock(ContainerExecutor.class));
Configuration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.DEBUG_NM_DELETE_DELAY_SEC, 60);
del.init(conf);
del.start();
try {
del.init(conf);
del.start();
del.delete("dingo", new Path("/does/not/exist"));
} finally {
del.stop();

View File

@ -73,7 +73,8 @@ public class TestLocalDirsHandlerService {
Assert.fail("Service should have thrown an exception due to wrong URI");
} catch (YarnRuntimeException e) {
}
Assert.assertTrue("Service should not be inited", dirSvc.getServiceState()
.compareTo(STATE.NOTINITED) == 0);
Assert.assertEquals("Service should not be inited",
STATE.STOPPED,
dirSvc.getServiceState());
}
}

View File

@ -49,6 +49,8 @@ public class TestNodeManager {
} catch (YarnRuntimeException e) {
//PASS
assert(e.getCause().getMessage().contains("dummy executor init called"));
} finally {
nm.stop();
}
}

View File

@ -83,6 +83,7 @@ public class TestNodeManagerShutdown {
static final String user = "nobody";
private FileContext localFS;
private ContainerId cId;
private NodeManager nm;
@Before
public void setup() throws UnsupportedFileSystemException {
@ -98,13 +99,16 @@ public class TestNodeManagerShutdown {
@After
public void tearDown() throws IOException, InterruptedException {
if (nm != null) {
nm.stop();
}
localFS.delete(new Path(basedir.getPath()), true);
}
@Test
public void testKillContainersOnShutdown() throws IOException,
YarnException {
NodeManager nm = new TestNodeManager();
nm = new TestNodeManager();
nm.init(createNMConfig());
nm.start();
startContainer(nm, cId, localFS, tmpDir, processStartFile);

View File

@ -33,6 +33,7 @@ import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.ConcurrentSkipListMap;
import java.util.concurrent.CyclicBarrier;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
import org.apache.commons.logging.Log;
@ -78,8 +79,8 @@ import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecret
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
import org.apache.hadoop.yarn.service.Service;
import org.apache.hadoop.yarn.service.Service.STATE;
import org.apache.hadoop.yarn.service.ServiceOperations;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
@ -110,9 +111,7 @@ public class TestNodeStatusUpdater {
public void tearDown() {
this.registeredNodes.clear();
heartBeatID = 0;
if (nm != null && nm.getServiceState() == STATE.STARTED) {
nm.stop();
}
ServiceOperations.stop(nm);
DefaultMetricsSystem.shutdown();
}
@ -316,9 +315,11 @@ public class TestNodeStatusUpdater {
public ResourceTracker resourceTracker =
new MyResourceTracker(this.context);
private Context context;
private final long waitStartTime;
private long waitStartTime;
private final long rmStartIntervalMS;
private final boolean rmNeverStart;
private volatile boolean triggered = false;
private long durationWhenTriggered = -1;
public MyNodeStatusUpdater4(Context context, Dispatcher dispatcher,
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics,
@ -330,15 +331,51 @@ public class TestNodeStatusUpdater {
this.rmNeverStart = rmNeverStart;
}
@Override
protected void serviceStart() throws Exception {
//record the startup time
this.waitStartTime = System.currentTimeMillis();
super.serviceStart();
}
@Override
protected ResourceTracker getRMClient() {
if(System.currentTimeMillis() - waitStartTime <= rmStartIntervalMS
|| rmNeverStart) {
throw new YarnRuntimeException("Faking RM start failure as start " +
"delay timer has not expired.");
} else {
return resourceTracker;
if (!triggered) {
long t = System.currentTimeMillis();
long duration = t - waitStartTime;
if (duration <= rmStartIntervalMS
|| rmNeverStart) {
throw new YarnRuntimeException("Faking RM start failure as start " +
"delay timer has not expired.");
} else {
//triggering
triggered = true;
durationWhenTriggered = duration;
}
}
return resourceTracker;
}
private boolean isTriggered() {
return triggered;
}
private long getWaitStartTime() {
return waitStartTime;
}
private long getDurationWhenTriggered() {
return durationWhenTriggered;
}
@Override
public String toString() {
return "MyNodeStatusUpdater4{" +
"rmNeverStart=" + rmNeverStart +
", triggered=" + triggered +
", duration=" + durationWhenTriggered +
", rmStartIntervalMS=" + rmStartIntervalMS +
'}';
}
}
@ -390,13 +427,10 @@ public class TestNodeStatusUpdater {
}
@Override
public void stop() {
super.stop();
protected void serviceStop() throws Exception {
super.serviceStop();
isStopped = true;
try {
syncBarrier.await();
} catch (Exception e) {
}
syncBarrier.await(10000, TimeUnit.MILLISECONDS);
}
}
//
@ -580,7 +614,9 @@ public class TestNodeStatusUpdater {
nodeStatus.setResponseId(heartBeatID);
NodeHeartbeatResponse nhResponse =
YarnServerBuilderUtils.newNodeHeartbeatResponse(heartBeatID,
heartBeatNodeAction, null, null, null, 1000L);
heartBeatNodeAction,
null, null, null,
1000L);
return nhResponse;
}
}
@ -749,12 +785,35 @@ public class TestNodeStatusUpdater {
Assert.assertEquals(STATE.STOPPED, nm.getServiceState());
}
@Test
public void testNMShutdownForRegistrationFailure() {
private abstract class NodeManagerWithCustomNodeStatusUpdater extends NodeManager {
private NodeStatusUpdater updater;
nm = new NodeManager() {
private NodeManagerWithCustomNodeStatusUpdater() {
}
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher,
NodeHealthCheckerService healthChecker) {
updater = createUpdater(context, dispatcher, healthChecker);
return updater;
}
public NodeStatusUpdater getUpdater() {
return updater;
}
abstract NodeStatusUpdater createUpdater(Context context,
Dispatcher dispatcher,
NodeHealthCheckerService healthChecker);
}
@Test
public void testNMShutdownForRegistrationFailure() throws Exception {
nm = new NodeManagerWithCustomNodeStatusUpdater() {
@Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
protected NodeStatusUpdater createUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
MyNodeStatusUpdater nodeStatusUpdater = new MyNodeStatusUpdater(
context, dispatcher, healthChecker, metrics);
@ -765,14 +824,14 @@ public class TestNodeStatusUpdater {
return nodeStatusUpdater;
}
};
verifyNodeStartFailure("org.apache.hadoop.yarn.YarnRuntimeException: "
+ "Recieved SHUTDOWN signal from Resourcemanager ,"
verifyNodeStartFailure(
"Recieved SHUTDOWN signal from Resourcemanager ,"
+ "Registration of NodeManager failed, "
+ "Message from ResourceManager: RM Shutting Down Node");
}
@Test (timeout = 150000)
public void testNMConnectionToRM() {
public void testNMConnectionToRM() throws Exception {
final long delta = 50000;
final long connectionWaitSecs = 5;
final long connectionRetryIntervalSecs = 1;
@ -786,9 +845,10 @@ public class TestNodeStatusUpdater {
connectionRetryIntervalSecs);
//Test NM try to connect to RM Several times, but finally fail
nm = new NodeManager() {
NodeManagerWithCustomNodeStatusUpdater nmWithUpdater;
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,
@ -802,19 +862,25 @@ public class TestNodeStatusUpdater {
nm.start();
Assert.fail("NM should have failed to start due to RM connect failure");
} catch(Exception e) {
Assert.assertTrue("NM should have tried re-connecting to RM during " +
long t = System.currentTimeMillis();
long duration = t - waitStartTime;
boolean waitTimeValid = (duration >= 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 @@ public class TestNodeStatusUpdater {
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 @@ public class TestNodeStatusUpdater {
* 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 @@ public class TestNodeStatusUpdater {
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 class TestNodeStatusUpdater {
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 @@ public class TestNodeStatusUpdater {
}
}
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,

View File

@ -202,8 +202,7 @@ public abstract class BaseContainerManagerTest {
@After
public void tearDown() throws IOException, InterruptedException {
if (containerManager != null
&& containerManager.getServiceState() == STATE.STARTED) {
if (containerManager != null) {
containerManager.stop();
}
createContainerExecutor().deleteAsUser(user,

View File

@ -63,20 +63,20 @@ public class TestAuxServices {
}
public ArrayList<Integer> getAppIdsStopped() {
return (ArrayList)this.stoppedApps.clone();
return (ArrayList<Integer>)this.stoppedApps.clone();
}
@Override
public void init(Configuration conf) {
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) {

View File

@ -237,7 +237,7 @@ public class TestResourceLocalizationService {
dirsHandler.init(conf);
DeletionService delService = new DeletionService(exec);
delService.init(null);
delService.init(new Configuration());
delService.start();
ResourceLocalizationService rawService =

View File

@ -104,21 +104,32 @@ public class TestNMWebServer {
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

View File

@ -92,8 +92,7 @@ public class AdminService extends AbstractService implements RMAdminProtocol {
}
@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 class AdminService extends AbstractService implements RMAdminProtocol {
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 class AdminService extends AbstractService implements RMAdminProtocol {
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 {

View File

@ -107,7 +107,7 @@ public class ApplicationMasterService extends AbstractService implements
}
@Override
public void start() {
protected void serviceStart() throws Exception {
Configuration conf = getConfig();
YarnRPC rpc = YarnRPC.create(conf);
@ -133,7 +133,7 @@ public class ApplicationMasterService extends AbstractService implements
this.bindAddress =
conf.updateConnectAddr(YarnConfiguration.RM_SCHEDULER_ADDRESS,
server.getListenerAddress());
super.start();
super.serviceStart();
}
@Private
@ -447,10 +447,10 @@ public class ApplicationMasterService extends AbstractService implements
}
@Override
public void stop() {
protected void serviceStop() throws Exception {
if (this.server != null) {
this.server.stop();
}
super.stop();
super.serviceStop();
}
}

View File

@ -128,13 +128,13 @@ public class ClientRMService extends AbstractService implements
}
@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 class ClientRMService extends AbstractService implements
// 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) {

View File

@ -37,12 +37,12 @@ public class NMLivelinessMonitor extends AbstractLivelinessMonitor<NodeId> {
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

View File

@ -57,7 +57,7 @@ public class NodesListManager extends AbstractService implements
}
@Override
public void init(Configuration conf) {
protected void serviceInit(Configuration conf) throws Exception {
this.conf = conf;
@ -83,7 +83,7 @@ public class NodesListManager extends AbstractService implements
throw new YarnRuntimeException(ioe2);
}
}
super.init(conf);
super.serviceInit(conf);
}
private void printConfiguredHosts() {

View File

@ -137,7 +137,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
}
@Override
public synchronized void init(Configuration conf) {
protected void serviceInit(Configuration conf) throws Exception {
validateConfigs(conf);
@ -257,7 +257,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
new RMNMInfo(this.rmContext, this.scheduler);
super.init(conf);
super.serviceInit(conf);
}
@VisibleForTesting
@ -398,17 +398,17 @@ public class ResourceManager extends CompositeService implements Recoverable {
}
@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 class ResourceManager extends CompositeService implements Recoverable {
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
this.stopped = true;
this.eventProcessor.interrupt();
try {
@ -456,7 +456,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
} catch (InterruptedException e) {
throw new YarnRuntimeException(e);
}
super.stop();
super.serviceStop();
}
@Override
@ -577,7 +577,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
}
@Override
public void start() {
protected void serviceStart() throws Exception {
try {
doSecureLogin();
} catch(IOException ie) {
@ -616,7 +616,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
conf.set(YarnConfiguration.RM_WEBAPP_ADDRESS, resolvedAddress);
}
super.start();
super.serviceStart();
/*synchronized(shutdown) {
try {
@ -635,14 +635,20 @@ public class ResourceManager extends CompositeService implements Recoverable {
}
@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 class ResourceManager extends CompositeService implements Recoverable {
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() {

View File

@ -99,7 +99,7 @@ public class ResourceTrackerService extends AbstractService implements
}
@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 class ResourceTrackerService extends AbstractService implements
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 class ResourceTrackerService extends AbstractService implements
}
@Override
public synchronized void stop() {
protected void serviceStop() throws Exception {
if (this.server != null) {
this.server.stop();
}
super.stop();
super.serviceStop();
}
@SuppressWarnings("unchecked")

View File

@ -50,9 +50,10 @@ public class ApplicationMasterLauncher extends AbstractService implements
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 @@ public class ApplicationMasterLauncher extends AbstractService implements
}
public void stop() {
@Override
protected void serviceStop() throws Exception {
launcherHandlingThread.interrupt();
try {
launcherHandlingThread.join();
@ -77,7 +79,6 @@ public class ApplicationMasterLauncher extends AbstractService implements
LOG.info(launcherHandlingThread.getName() + " interrupted during join ",
ie); }
launcherPool.shutdown();
super.stop();
}
private class LauncherThread extends Thread {

View File

@ -35,8 +35,8 @@ public class AMLivelinessMonitor extends AbstractLivelinessMonitor<ApplicationAt
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);

View File

@ -38,13 +38,13 @@ public class ContainerAllocationExpirer extends
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

View File

@ -88,19 +88,18 @@ public class DelegationTokenRenewer extends AbstractService {
}
@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 class DelegationTokenRenewer extends AbstractService {
"DelayedTokenCanceller");
delayedRemovalThread.start();
}
super.serviceStart();
}
@Override
public synchronized void stop() {
protected void serviceStop() {
if (renewalTimer != null) {
renewalTimer.cancel();
}
@ -133,8 +133,6 @@ public class DelegationTokenRenewer extends AbstractService {
LOG.info("Interrupted while joining on delayed removal thread.", e);
}
}
super.stop();
}
/**

View File

@ -293,12 +293,12 @@ public class MockRM extends ResourceManager {
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 @@ public class MockRM extends ResourceManager {
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 class MockRM extends ResourceManager {
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 class MockRM extends ResourceManager {
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 class MockRM extends ResourceManager {
}
@Override
public void stop() {
protected void serviceStop() {
// don't do anything
}
};
@ -367,12 +367,12 @@ public class MockRM extends ResourceManager {
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
}
};

Some files were not shown because too many files have changed in this diff Show More