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:
parent
5244bf6860
commit
0928502029
|
@ -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
|
||||
|
|
|
@ -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" />
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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: ");
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -350,7 +350,7 @@ class JHEvenHandlerForTest extends JobHistoryEventHandler {
|
|||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
protected void serviceStart() {
|
||||
}
|
||||
|
||||
@Override
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
};
|
||||
}
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
};
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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>
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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;
|
||||
}
|
|
@ -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());
|
||||
}
|
||||
}
|
|
@ -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->description values
|
||||
*/
|
||||
public Map<String, String> getBlockers();
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
|
||||
}
|
|
@ -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);
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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 + "]";
|
||||
}
|
||||
}
|
|
@ -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);
|
||||
}
|
||||
|
||||
}
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
|
@ -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() {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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.
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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());
|
||||
}
|
||||
}
|
||||
|
|
|
@ -49,6 +49,8 @@ public class TestNodeManager {
|
|||
} catch (YarnRuntimeException e) {
|
||||
//PASS
|
||||
assert(e.getCause().getMessage().contains("dummy executor init called"));
|
||||
} finally {
|
||||
nm.stop();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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 =
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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) {
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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() {
|
||||
|
|
|
@ -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")
|
||||
|
|
|
@ -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 {
|
||||
|
|
|
@ -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);
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
/**
|
||||
|
|
|
@ -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
Loading…
Reference in New Issue