[
https://issues.apache.org/jira/browse/YARN-117?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=13627215#comment-13627215
]
Steve Loughran commented on YARN-117:
-------------------------------------
one test failing, looks like a service isn't shutting down reliably, so the
next test in the same JVM can't bind to a port.
{code}
org.apache.hadoop.yarn.YarnException: java.net.BindException: Problem binding
to [localhost:12345] java.net.BindException: Address already in use; For more
details see: http://wiki.apache.org/hadoop/BindException
at
org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl.getServer(RpcServerFactoryPBImpl.java:139)
at
org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC.getServer(HadoopYarnProtoRPC.java:63)
at org.apache.hadoop.yarn.ipc.YarnRPC.getServer(YarnRPC.java:52)
at
org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl.innerStart(ContainerManagerImpl.java:230)
at
org.apache.hadoop.yarn.service.AbstractService.start(AbstractService.java:175)
at
org.apache.hadoop.yarn.service.CompositeService.innerStart(CompositeService.java:76)
at
org.apache.hadoop.yarn.server.nodemanager.NodeManager.innerStart(NodeManager.java:209)
at
org.apache.hadoop.yarn.service.AbstractService.start(AbstractService.java:175)
at
org.apache.hadoop.yarn.server.nodemanager.TestNodeStatusUpdater.testNodeReboot(TestNodeStatusUpdater.java:726)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at
org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:44)
at
org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:15)
at
org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:41)
at
org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:20)
at
org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:28)
at
org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:31)
at
org.junit.runners.BlockJUnit4ClassRunner.runNotIgnored(BlockJUnit4ClassRunner.java:79)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:71)
at
org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:49)
at org.junit.runners.ParentRunner$3.run(ParentRunner.java:193)
at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:52)
at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:191)
at org.junit.runners.ParentRunner.access$000(ParentRunner.java:42)
at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:184)
at org.junit.runners.ParentRunner.run(ParentRunner.java:236)
at
org.apache.maven.surefire.junit4.JUnit4Provider.execute(JUnit4Provider.java:252)
at
org.apache.maven.surefire.junit4.JUnit4Provider.executeTestSet(JUnit4Provider.java:141)
at
org.apache.maven.surefire.junit4.JUnit4Provider.invoke(JUnit4Provider.java:112)
at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
at
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:39)
at
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:25)
at java.lang.reflect.Method.invoke(Method.java:597)
at
org.apache.maven.surefire.util.ReflectionUtils.invokeMethodWithArray(ReflectionUtils.java:189)
at
org.apache.maven.surefire.booter.ProviderFactory$ProviderProxy.invoke(ProviderFactory.java:165)
at
org.apache.maven.surefire.booter.ProviderFactory.invokeProvider(ProviderFactory.java:85)
at
org.apache.maven.surefire.booter.ForkedBooter.runSuitesInProcess(ForkedBooter.java:115)
at
org.apache.maven.surefire.booter.ForkedBooter.main(ForkedBooter.java:75)
Caused by: java.net.BindException: Problem binding to [localhost:12345]
java.net.BindException: Address already in use; For more details see:
http://wiki.apache.org/hadoop/BindException
at org.apache.hadoop.net.NetUtils.wrapException(NetUtils.java:716)
at org.apache.hadoop.ipc.Server.bind(Server.java:415)
at org.apache.hadoop.ipc.Server$Listener.<init>(Server.java:518)
at org.apache.hadoop.ipc.Server.<init>(Server.java:1963)
at org.apache.hadoop.ipc.RPC$Server.<init>(RPC.java:986)
at
org.apache.hadoop.ipc.ProtobufRpcEngine$Server.<init>(ProtobufRpcEngine.java:427)
at
org.apache.hadoop.ipc.ProtobufRpcEngine.getServer(ProtobufRpcEngine.java:402)
at org.apache.hadoop.ipc.RPC$Builder.build(RPC.java:829)
at
org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl.createServer(RpcServerFactoryPBImpl.java:169)
at
org.apache.hadoop.yarn.factories.impl.pb.RpcServerFactoryPBImpl.getServer(RpcServerFactoryPBImpl.java:132)
... 39 more
{code}
> Enhance YARN service model
> --------------------------
>
> Key: YARN-117
> URL: https://issues.apache.org/jira/browse/YARN-117
> Project: Hadoop YARN
> Issue Type: Improvement
> Reporter: Steve Loughran
> Assignee: Steve Loughran
> Attachments: YARN-117-2.patch, YARN-117.patch
>
>
> Having played the YARN service model, there are some issues
> that I've identified based on past work and initial use.
> This JIRA issue is an overall one to cover the issues, with solutions pushed
> out to separate JIRAs.
> h2. state model prevents stopped state being entered if you could not
> successfully start the service.
> In the current lifecycle you cannot stop a service unless it was successfully
> started, but
> * {{init()}} may acquire resources that need to be explicitly released
> * if the {{start()}} operation fails partway through, the {{stop()}}
> operation may be needed to release resources.
> *Fix:* make {{stop()}} a valid state transition from all states and require
> the implementations to be able to stop safely without requiring all fields to
> be non null.
> Before anyone points out that the {{stop()}} operations assume that all
> fields are valid; and if called before a {{start()}} they will NPE;
> MAPREDUCE-3431 shows that this problem arises today, MAPREDUCE-3502 is a fix
> for this. It is independent of the rest of the issues in this doc but it will
> aid making {{stop()}} execute from all states other than "stopped".
> MAPREDUCE-3502 is too big a patch and needs to be broken down for easier
> review and take up; this can be done with issues linked to this one.
> h2. AbstractService doesn't prevent duplicate state change requests.
> The {{ensureState()}} checks to verify whether or not a state transition is
> allowed from the current state are performed in the base {{AbstractService}}
> class -yet subclasses tend to call this *after* their own {{init()}},
> {{start()}} & {{stop()}} operations. This means that these operations can be
> performed out of order, and even if the outcome of the call is an exception,
> all actions performed by the subclasses will have taken place. MAPREDUCE-3877
> demonstrates this.
> This is a tricky one to address. In HADOOP-3128 I used a base class instead
> of an interface and made the {{init()}}, {{start()}} & {{stop()}} methods
> {{final}}. These methods would do the checks, and then invoke protected inner
> methods, {{innerStart()}}, {{innerStop()}}, etc. It should be possible to
> retrofit the same behaviour to everything that extends {{AbstractService}}
> -something that must be done before the class is considered stable (because
> once the lifecycle methods are declared final, all subclasses that are out of
> the source tree will need fixing by the respective developers.
> h2. AbstractService state change doesn't defend against race conditions.
> There's no concurrency locks on the state transitions. Whatever fix for wrong
> state calls is added should correct this to prevent re-entrancy, such as
> {{stop()}} being called from two threads.
> h2. Static methods to choreograph of lifecycle operations
> Helper methods to move things through lifecycles. init->start is common,
> stop-if-service!=null another. Some static methods can execute these, and
> even call {{stop()}} if {{init()}} raises an exception. These could go into a
> class {{ServiceOps}} in the same package. These can be used by those services
> that wrap other services, and help manage more robust shutdowns.
> h2. state transition failures are something that registered service listeners
> may wish to be informed of.
> When a state transition fails a {{RuntimeException}} can be thrown -and the
> service listeners are not informed as the notification point isn't reached.
> They may wish to know this, especially for management and diagnostics.
> *Fix:* extend {{ServiceStateChangeListener}} with a callback such as
> {{stateChangeFailed(Service service,Service.State targeted-state,
> RuntimeException e)}} that is invoked from the (final) state change methods
> in the {{AbstractService}} class (once they delegate to their inner
> {{innerStart()}}, {{innerStop()}} methods; make a no-op on the existing
> implementations of the interface.
> h2. Service listener failures not handled
> Is this an error an error or not? Log and ignore may not be what is desired.
> *Proposed:* during {{stop()}} any exception by a listener is caught and
> discarded, to increase the likelihood of a better shutdown, but do not add
> try-catch clauses to the other state changes.
> h2. Support static listeners for all AbstractServices
> Add support to {{AbstractService}} that allow callers to register listeners
> for all instances. The existing listener interface could be used. This allows
> management tools to hook into the events.
> The static listeners would be invoked for all state changes except creation
> (base class shouldn't be handing out references to itself at this point).
> These static events could all be async, pushed through a shared
> {{ConcurrentLinkedQueue}}; failures logged at warn and the rest of the
> listeners invoked.
> h2. Add some example listeners for management/diagnostics
> * event to commons log for humans.
> * events for machines hooked up to the JSON logger.
> * for testing: something that be told to fail.
> h2. Services should support signal interruptibility
> The services would benefit from a way of shutting them down on a kill signal;
> this can be done via a runtime hook. It should not be automatic though, as
> composite services will get into a very complex state during shutdown. Better
> to provide a hook that lets you register/unregister services to terminate,
> and have the relevant {{main()}} entry points tell their root services to
> register themselves.
--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira