YARN-746. Renamed Service.register() and Service.unregister() to registerServiceListener() & unregisterServiceListener() respectively. Contributed by Steve Loughran.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1492780 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-06-13 17:49:14 +00:00
parent 0928502029
commit 686ac311c9
7 changed files with 34 additions and 27 deletions

View File

@ -136,6 +136,10 @@ Release 2.1.0-beta - UNRELEASED
robust subclassing and migrated yarn-common services. (Steve Loughran via
vinodkv)
YARN-746. Renamed Service.register() and Service.unregister() to
registerServiceListener() & unregisterServiceListener() respectively.
(Steve Loughran via vinodkv)
NEW FEATURES
YARN-482. FS: Extend SchedulingMode to intermediate queues.

View File

@ -353,12 +353,12 @@ public abstract class AbstractService implements Service {
}
@Override
public void register(ServiceStateChangeListener l) {
public void registerServiceListener(ServiceStateChangeListener l) {
listeners.add(l);
}
@Override
public void unregister(ServiceStateChangeListener l) {
public void unregisterServiceListener(ServiceStateChangeListener l) {
listeners.remove(l);
}

View File

@ -54,13 +54,13 @@ public class FilterService implements Service {
}
@Override
public void register(ServiceStateChangeListener listener) {
service.register(listener);
public void registerServiceListener(ServiceStateChangeListener listener) {
service.registerServiceListener(listener);
}
@Override
public void unregister(ServiceStateChangeListener listener) {
service.unregister(listener);
public void unregisterServiceListener(ServiceStateChangeListener listener) {
service.unregisterServiceListener(listener);
}
@Override

View File

@ -129,16 +129,19 @@ public interface Service extends Closeable {
void close() throws IOException;
/**
* Register an instance of the service state change events.
* Register a listener to the service state change events.
* If the supplied listener is already listening to this service,
* this method is a no-op.
* @param listener a new listener
*/
void register(ServiceStateChangeListener listener);
void registerServiceListener(ServiceStateChangeListener listener);
/**
* Unregister a previously instance of the service state change events.
* Unregister a previously registered listener of the service state
* change events. No-op if the listener is already unregistered.
* @param listener the listener to unregister.
*/
void unregister(ServiceStateChangeListener listener);
void unregisterServiceListener(ServiceStateChangeListener listener);
/**
* Get the name of this service.

View File

@ -186,7 +186,7 @@ public class TestServiceLifecycle extends ServiceAssert {
@Test
public void testStopFailingInitAndStop() throws Throwable {
BreakableService svc = new BreakableService(true, false, true);
svc.register(new LoggingStateChangeListener());
svc.registerServiceListener(new LoggingStateChangeListener());
try {
svc.init(new Configuration());
fail("Expected a failure, got " + svc);
@ -220,7 +220,7 @@ public class TestServiceLifecycle extends ServiceAssert {
public void testServiceNotifications() throws Throwable {
BreakableService svc = new BreakableService(false, false, false);
BreakableStateChangeListener listener = new BreakableStateChangeListener();
svc.register(listener);
svc.registerServiceListener(listener);
svc.init(new Configuration());
assertEventCount(listener, 1);
svc.start();
@ -239,10 +239,10 @@ public class TestServiceLifecycle extends ServiceAssert {
public void testServiceNotificationsStopOnceUnregistered() throws Throwable {
BreakableService svc = new BreakableService(false, false, false);
BreakableStateChangeListener listener = new BreakableStateChangeListener();
svc.register(listener);
svc.registerServiceListener(listener);
svc.init(new Configuration());
assertEventCount(listener, 1);
svc.unregister(listener);
svc.unregisterServiceListener(listener);
svc.start();
assertEventCount(listener, 1);
svc.stop();
@ -267,12 +267,12 @@ public class TestServiceLifecycle extends ServiceAssert {
new SelfUnregisteringBreakableStateChangeListener();
BreakableStateChangeListener l2 =
new BreakableStateChangeListener();
svc.register(listener);
svc.register(l2);
svc.registerServiceListener(listener);
svc.registerServiceListener(l2);
svc.init(new Configuration());
assertEventCount(listener, 1);
assertEventCount(l2, 1);
svc.unregister(listener);
svc.unregisterServiceListener(listener);
svc.start();
assertEventCount(listener, 1);
assertEventCount(l2, 2);
@ -287,7 +287,7 @@ public class TestServiceLifecycle extends ServiceAssert {
@Override
public synchronized void stateChanged(Service service) {
super.stateChanged(service);
service.unregister(this);
service.unregisterServiceListener(this);
}
}
@ -301,7 +301,7 @@ public class TestServiceLifecycle extends ServiceAssert {
BreakableService svc = new BreakableService(false, false, false);
BreakableStateChangeListener listener = new BreakableStateChangeListener();
listener.setFailingState(Service.STATE.STARTED);
svc.register(listener);
svc.registerServiceListener(listener);
svc.init(new Configuration());
assertEventCount(listener, 1);
//start this; the listener failed but this won't show
@ -326,7 +326,7 @@ public class TestServiceLifecycle extends ServiceAssert {
//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.registerServiceListener(listener);
service.init(new Configuration());
service.start();
assertServiceInState(service, Service.STATE.STARTED);
@ -344,7 +344,7 @@ public class TestServiceLifecycle extends ServiceAssert {
public void testSelfTerminatingService() throws Throwable {
SelfTerminatingService service = new SelfTerminatingService();
BreakableStateChangeListener listener = new BreakableStateChangeListener();
service.register(listener);
service.registerServiceListener(listener);
service.init(new Configuration());
assertEventCount(listener, 1);
//start the service
@ -357,7 +357,7 @@ public class TestServiceLifecycle extends ServiceAssert {
public void testStartInInitService() throws Throwable {
Service service = new StartInInitService();
BreakableStateChangeListener listener = new BreakableStateChangeListener();
service.register(listener);
service.registerServiceListener(listener);
service.init(new Configuration());
assertServiceInState(service, Service.STATE.STARTED);
assertEventCount(listener, 1);
@ -367,7 +367,7 @@ public class TestServiceLifecycle extends ServiceAssert {
public void testStopInInitService() throws Throwable {
Service service = new StopInInitService();
BreakableStateChangeListener listener = new BreakableStateChangeListener();
service.register(listener);
service.registerServiceListener(listener);
service.init(new Configuration());
assertServiceInState(service, Service.STATE.STOPPED);
assertEventCount(listener, 1);

View File

@ -120,7 +120,7 @@ public class AuxServices extends AbstractService
AuxiliaryService service = entry.getValue();
String name = entry.getKey();
service.start();
service.register(this);
service.registerServiceListener(this);
ByteBuffer meta = service.getMeta();
if(meta != null) {
serviceMeta.put(name, meta);
@ -135,7 +135,7 @@ public class AuxServices extends AbstractService
synchronized (serviceMap) {
for (Service service : serviceMap.values()) {
if (service.getServiceState() == Service.STATE.STARTED) {
service.unregister(this);
service.unregisterServiceListener(this);
service.stop();
}
}

View File

@ -156,7 +156,7 @@ public class ContainerManagerImpl extends CompositeService implements
// Start configurable services
auxiliaryServices = new AuxServices();
auxiliaryServices.register(this);
auxiliaryServices.registerServiceListener(this);
addService(auxiliaryServices);
this.containersMonitor =
@ -265,7 +265,7 @@ public class ContainerManagerImpl extends CompositeService implements
@Override
public void serviceStop() throws Exception {
if (auxiliaryServices.getServiceState() == STARTED) {
auxiliaryServices.unregister(this);
auxiliaryServices.unregisterServiceListener(this);
}
if (server != null) {
server.stop();