You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@reef.apache.org by Sergiy Matusevych <se...@gmail.com> on 2017/01/24 03:50:37 UTC

YARN Client and Unmanaged AM running in the same process?

Hi fellow YARN developers,

I am writing a YARN application that runs both Client *and* Unmanaged
Application Master in the same JVM process. I have a toy example that
starts a YARN application and the AM in Unmanaged mode, and then just shuts
it down:

https://github.com/apache/reef/blob/master/lang/java/reef-runtime-yarn/src/test/java/org/apache/reef/runtime/yarn/driver/unmanaged/UnmanagedAmTest.java

(I wrapped it as a unit test, but the code is 100% independent of REEF, so
you can copy & paste it if you want to play with it; I can also build a
small maven project around it).

The app *almost* works - the problem seems to be that the call on line 117

    rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
"Success!", null);

fails to update the status of the application on the Resource Manager side.
I would expect it to be FINISHED/SUCCEDED, but instead it stays in
RUNNING/UNDEFINED well after the client/AM process terminates, and
eventually got marked as FAILED/FAILED by the RM.

Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
issue that seems to be related to the problem:

https://issues.apache.org/jira/browse/YARN-273
"Add an unmanaged AM client for in-process AMs"

Can someone confirm if my problem is indeed related to that issue, or is
there something wrong with my code?

Thank you!
Sergiy.

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Cool! I'll try it on Hadoop 2.7.3.

Thanks a lot for your help Jason!
-- Sergiy.


On Tue, Jan 24, 2017 at 2:59 PM, Jason Lowe <jl...@yahoo-inc.com> wrote:

> Version 2.7.1.2.3.0.0-2557 is not a standard Apache Hadoop version, nor is
> the specified git hash present in the Apache Hadoop repo.  I cannot match
> the code exactly with what you are running.
>
> What is happening according to the RM log is this sequence:
> - The RM is trying to publish system metrics corresponding to the AM
> registration and that blows up with a NullPointerException
> - Because of that NPE the app attempt state machine never properly
> transitions out of the the LAUNCHED state
> - When the AM tries to unregister it is an error because the app is still
> in the LAUNCHED state (due to the incomplete registration)
>
> This looks like the same problem described in YARN-4452
> <https://issues.apache.org/jira/browse/YARN-4452>.  The version you are
> reporting implies it is based on Apache Hadoop 2.7.1, and that problem was
> fixed in 2.7.3.  Check with your vendor and see if that fix is included in
> your version.  I am guessing that it is not and adding that patch will fix
> this use case for you.
>
> Jason
>
>
>
> On Tuesday, January 24, 2017 1:53 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
>
>
> Hi Jason,
>
> Here's what I see in our YARN logs (below). Could it be that I forget to
> populate some field in the registration or, say, application/container
> launch context?
>
> P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
> 9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
> 72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.
>
> Thanks a lot for your help!
>
> Cheers,
> Sergiy.
>
>
> 2017-01-23 19:39:40,544 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
> new applicationId: 1252
> 2017-01-23 19:39:40,575 WARN
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The
> specific
> max attempts: 0 for application: 1252 is invalid, because it is out of the
> range [1, 20]. Use the global max attempts instead.
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
> with id 1252 submitted by user sergiym
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
> application with id application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Submit Application Request
> TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW to NEW_SAVING
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
> Storing info for app: application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.ParentQueue:
> Application added - appId: application_1465994698013_1252 user: sergiym
> leaf-queue of parent: root #applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Accepted application application_1465994698013_1252 from user: sergiym, in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
> Registering app attempt : appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application application_1465994698013_1252 from user: sergiym activated in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application added - appId: application_1465994698013_1252 user:
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue$User@50ac15c8,
> leaf-queue: default #user-pending-applications: 0
> #user-active-applications: 1 #queue-pending-applications: 0
> #queue-active-applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Added Application Attempt appattempt_1465994698013_1252_000001 to
> scheduler
> from user sergiym in queue default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Storing attempt: AppId: application_1465994698013_1252 AttemptId:
> appattempt_1465994698013_1252_000001 MasterContainer: null
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from SUBMITTED to
> LAUNCHED_UNMANAGED_SAVING
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Create AMRMToken for ApplicationAttempt:
> appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Creating password for appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from
> LAUNCHED_UNMANAGED_SAVING to LAUNCHED
> 2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
> Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
> registration appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Register App Master
> TARGET=ApplicationMasterService RESULT=SUCCESS
> APPID=application_1465994698013_1252
> APPATTEMPTID=appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
> handling event type REGISTERED for applicationAttempt
> application_1465994698013_1252
> java.lang.NullPointerException
> at
> org.apache.hadoop.yarn.server.resourcemanager.metrics.
> SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:
> 145)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1405)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1380)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.
> doTransition(StateMachineFactory.java:362)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:302)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from ACCEPTED to RUNNING
> 2017-01-23 19:39:42,344 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> STATUS_UPDATE at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,407 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> UNREGISTERED at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8032: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8030: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:50:59,224 INFO
> org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
> Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs
>
>
> On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
> >
> > Have you looked at the ResourceManager logs to see what it is doing when
> it receives the unregister request?  I'm wondering if there's an exception
> or error that could explain why it is not working as expected.  The sample
> code works against trunk for me.  The unmanaged AM registered,
> unregistered, and the final status of the application on the RM was
> FINISHED/SUCCEEDED.
> >
> > Jason
> >
> >
> > On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
> >
> >
> > Hi fellow YARN developers,
> >
> > I am writing a YARN application that runs both Client *and* Unmanaged
> > Application Master in the same JVM process. I have a toy example that
> > starts a YARN application and the AM in Unmanaged mode, and then just
> shuts
> > it down:
> >
> >
> https://github.com/apache/reef/blob/master/lang/java/
> reef-runtime-yarn/src/test/java/org/apache/reef/runtime/
> yarn/driver/unmanaged/UnmanagedAmTest.java
> >
> > (I wrapped it as a unit test, but the code is 100% independent of REEF,
> so
> > you can copy & paste it if you want to play with it; I can also build a
> > small maven project around it).
> >
> > The app *almost* works - the problem seems to be that the call on line
> 117
> >
> >    rmClient.unregisterApplicationMaster(FinalApplicationStatus.
> SUCCEEDED,
> > "Success!", null);
> >
> > fails to update the status of the application on the Resource Manager
> side.
> > I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> > RUNNING/UNDEFINED well after the client/AM process terminates, and
> > eventually got marked as FAILED/FAILED by the RM.
> >
> > Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> > issue that seems to be related to the problem:
> >
> > https://issues.apache.org/jira/browse/YARN-273
> > "Add an unmanaged AM client for in-process AMs"
> >
> > Can someone confirm if my problem is indeed related to that issue, or is
> > there something wrong with my code?
> >
> > Thank you!
> > Sergiy.
> >
> >
>
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Hi Jason,

Thanks a lot for your help! I just wanted to confirm that my problem **was**
 YARN-4452 <https://issues.apache.org/jira/browse/YARN-4452> and it had
been resolved after upgrade to Hadoop 2.7.3

Thank you,
Sergiy.


On Tue, Jan 24, 2017 at 2:59 PM, Jason Lowe <jl...@yahoo-inc.com> wrote:

> Version 2.7.1.2.3.0.0-2557 is not a standard Apache Hadoop version, nor is
> the specified git hash present in the Apache Hadoop repo.  I cannot match
> the code exactly with what you are running.
>
> What is happening according to the RM log is this sequence:
> - The RM is trying to publish system metrics corresponding to the AM
> registration and that blows up with a NullPointerException
> - Because of that NPE the app attempt state machine never properly
> transitions out of the the LAUNCHED state
> - When the AM tries to unregister it is an error because the app is still
> in the LAUNCHED state (due to the incomplete registration)
>
> This looks like the same problem described in YARN-4452
> <https://issues.apache.org/jira/browse/YARN-4452>.  The version you are
> reporting implies it is based on Apache Hadoop 2.7.1, and that problem was
> fixed in 2.7.3.  Check with your vendor and see if that fix is included in
> your version.  I am guessing that it is not and adding that patch will fix
> this use case for you.
>
> Jason
>
>
>
> On Tuesday, January 24, 2017 1:53 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
>
>
> Hi Jason,
>
> Here's what I see in our YARN logs (below). Could it be that I forget to
> populate some field in the registration or, say, application/container
> launch context?
>
> P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
> 9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
> 72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.
>
> Thanks a lot for your help!
>
> Cheers,
> Sergiy.
>
>
> 2017-01-23 19:39:40,544 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
> new applicationId: 1252
> 2017-01-23 19:39:40,575 WARN
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The
> specific
> max attempts: 0 for application: 1252 is invalid, because it is out of the
> range [1, 20]. Use the global max attempts instead.
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
> with id 1252 submitted by user sergiym
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
> application with id application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Submit Application Request
> TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW to NEW_SAVING
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
> Storing info for app: application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.ParentQueue:
> Application added - appId: application_1465994698013_1252 user: sergiym
> leaf-queue of parent: root #applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Accepted application application_1465994698013_1252 from user: sergiym, in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
> Registering app attempt : appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application application_1465994698013_1252 from user: sergiym activated in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application added - appId: application_1465994698013_1252 user:
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue$User@50ac15c8,
> leaf-queue: default #user-pending-applications: 0
> #user-active-applications: 1 #queue-pending-applications: 0
> #queue-active-applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Added Application Attempt appattempt_1465994698013_1252_000001 to
> scheduler
> from user sergiym in queue default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Storing attempt: AppId: application_1465994698013_1252 AttemptId:
> appattempt_1465994698013_1252_000001 MasterContainer: null
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from SUBMITTED to
> LAUNCHED_UNMANAGED_SAVING
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Create AMRMToken for ApplicationAttempt:
> appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Creating password for appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from
> LAUNCHED_UNMANAGED_SAVING to LAUNCHED
> 2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
> Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
> registration appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Register App Master
> TARGET=ApplicationMasterService RESULT=SUCCESS
> APPID=application_1465994698013_1252
> APPATTEMPTID=appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
> handling event type REGISTERED for applicationAttempt
> application_1465994698013_1252
> java.lang.NullPointerException
> at
> org.apache.hadoop.yarn.server.resourcemanager.metrics.
> SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:
> 145)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1405)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1380)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.
> doTransition(StateMachineFactory.java:362)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:302)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from ACCEPTED to RUNNING
> 2017-01-23 19:39:42,344 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> STATUS_UPDATE at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,407 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> UNREGISTERED at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8032: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8030: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:50:59,224 INFO
> org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
> Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs
>
>
> On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
> >
> > Have you looked at the ResourceManager logs to see what it is doing when
> it receives the unregister request?  I'm wondering if there's an exception
> or error that could explain why it is not working as expected.  The sample
> code works against trunk for me.  The unmanaged AM registered,
> unregistered, and the final status of the application on the RM was
> FINISHED/SUCCEEDED.
> >
> > Jason
> >
> >
> > On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
> >
> >
> > Hi fellow YARN developers,
> >
> > I am writing a YARN application that runs both Client *and* Unmanaged
> > Application Master in the same JVM process. I have a toy example that
> > starts a YARN application and the AM in Unmanaged mode, and then just
> shuts
> > it down:
> >
> >
> https://github.com/apache/reef/blob/master/lang/java/
> reef-runtime-yarn/src/test/java/org/apache/reef/runtime/
> yarn/driver/unmanaged/UnmanagedAmTest.java
> >
> > (I wrapped it as a unit test, but the code is 100% independent of REEF,
> so
> > you can copy & paste it if you want to play with it; I can also build a
> > small maven project around it).
> >
> > The app *almost* works - the problem seems to be that the call on line
> 117
> >
> >    rmClient.unregisterApplicationMaster(FinalApplicationStatus.
> SUCCEEDED,
> > "Success!", null);
> >
> > fails to update the status of the application on the Resource Manager
> side.
> > I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> > RUNNING/UNDEFINED well after the client/AM process terminates, and
> > eventually got marked as FAILED/FAILED by the RM.
> >
> > Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> > issue that seems to be related to the problem:
> >
> > https://issues.apache.org/jira/browse/YARN-273
> > "Add an unmanaged AM client for in-process AMs"
> >
> > Can someone confirm if my problem is indeed related to that issue, or is
> > there something wrong with my code?
> >
> > Thank you!
> > Sergiy.
> >
> >
>
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Cool! I'll try it on Hadoop 2.7.3.

Thanks a lot for your help Jason!
-- Sergiy.


On Tue, Jan 24, 2017 at 2:59 PM, Jason Lowe <jl...@yahoo-inc.com> wrote:

> Version 2.7.1.2.3.0.0-2557 is not a standard Apache Hadoop version, nor is
> the specified git hash present in the Apache Hadoop repo.  I cannot match
> the code exactly with what you are running.
>
> What is happening according to the RM log is this sequence:
> - The RM is trying to publish system metrics corresponding to the AM
> registration and that blows up with a NullPointerException
> - Because of that NPE the app attempt state machine never properly
> transitions out of the the LAUNCHED state
> - When the AM tries to unregister it is an error because the app is still
> in the LAUNCHED state (due to the incomplete registration)
>
> This looks like the same problem described in YARN-4452
> <https://issues.apache.org/jira/browse/YARN-4452>.  The version you are
> reporting implies it is based on Apache Hadoop 2.7.1, and that problem was
> fixed in 2.7.3.  Check with your vendor and see if that fix is included in
> your version.  I am guessing that it is not and adding that patch will fix
> this use case for you.
>
> Jason
>
>
>
> On Tuesday, January 24, 2017 1:53 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
>
>
> Hi Jason,
>
> Here's what I see in our YARN logs (below). Could it be that I forget to
> populate some field in the registration or, say, application/container
> launch context?
>
> P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
> 9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
> 72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.
>
> Thanks a lot for your help!
>
> Cheers,
> Sergiy.
>
>
> 2017-01-23 19:39:40,544 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
> new applicationId: 1252
> 2017-01-23 19:39:40,575 WARN
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The
> specific
> max attempts: 0 for application: 1252 is invalid, because it is out of the
> range [1, 20]. Use the global max attempts instead.
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
> with id 1252 submitted by user sergiym
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
> application with id application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Submit Application Request
> TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW to NEW_SAVING
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
> Storing info for app: application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.ParentQueue:
> Application added - appId: application_1465994698013_1252 user: sergiym
> leaf-queue of parent: root #applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Accepted application application_1465994698013_1252 from user: sergiym, in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
> Registering app attempt : appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application application_1465994698013_1252 from user: sergiym activated in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application added - appId: application_1465994698013_1252 user:
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue$User@50ac15c8,
> leaf-queue: default #user-pending-applications: 0
> #user-active-applications: 1 #queue-pending-applications: 0
> #queue-active-applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Added Application Attempt appattempt_1465994698013_1252_000001 to
> scheduler
> from user sergiym in queue default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Storing attempt: AppId: application_1465994698013_1252 AttemptId:
> appattempt_1465994698013_1252_000001 MasterContainer: null
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from SUBMITTED to
> LAUNCHED_UNMANAGED_SAVING
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Create AMRMToken for ApplicationAttempt:
> appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Creating password for appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from
> LAUNCHED_UNMANAGED_SAVING to LAUNCHED
> 2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
> Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
> registration appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Register App Master
> TARGET=ApplicationMasterService RESULT=SUCCESS
> APPID=application_1465994698013_1252
> APPATTEMPTID=appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
> handling event type REGISTERED for applicationAttempt
> application_1465994698013_1252
> java.lang.NullPointerException
> at
> org.apache.hadoop.yarn.server.resourcemanager.metrics.
> SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:
> 145)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1405)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1380)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.
> doTransition(StateMachineFactory.java:362)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:302)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from ACCEPTED to RUNNING
> 2017-01-23 19:39:42,344 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> STATUS_UPDATE at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,407 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> UNREGISTERED at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8032: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8030: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:50:59,224 INFO
> org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
> Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs
>
>
> On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
> >
> > Have you looked at the ResourceManager logs to see what it is doing when
> it receives the unregister request?  I'm wondering if there's an exception
> or error that could explain why it is not working as expected.  The sample
> code works against trunk for me.  The unmanaged AM registered,
> unregistered, and the final status of the application on the RM was
> FINISHED/SUCCEEDED.
> >
> > Jason
> >
> >
> > On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
> >
> >
> > Hi fellow YARN developers,
> >
> > I am writing a YARN application that runs both Client *and* Unmanaged
> > Application Master in the same JVM process. I have a toy example that
> > starts a YARN application and the AM in Unmanaged mode, and then just
> shuts
> > it down:
> >
> >
> https://github.com/apache/reef/blob/master/lang/java/
> reef-runtime-yarn/src/test/java/org/apache/reef/runtime/
> yarn/driver/unmanaged/UnmanagedAmTest.java
> >
> > (I wrapped it as a unit test, but the code is 100% independent of REEF,
> so
> > you can copy & paste it if you want to play with it; I can also build a
> > small maven project around it).
> >
> > The app *almost* works - the problem seems to be that the call on line
> 117
> >
> >    rmClient.unregisterApplicationMaster(FinalApplicationStatus.
> SUCCEEDED,
> > "Success!", null);
> >
> > fails to update the status of the application on the Resource Manager
> side.
> > I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> > RUNNING/UNDEFINED well after the client/AM process terminates, and
> > eventually got marked as FAILED/FAILED by the RM.
> >
> > Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> > issue that seems to be related to the problem:
> >
> > https://issues.apache.org/jira/browse/YARN-273
> > "Add an unmanaged AM client for in-process AMs"
> >
> > Can someone confirm if my problem is indeed related to that issue, or is
> > there something wrong with my code?
> >
> > Thank you!
> > Sergiy.
> >
> >
>
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Hi Jason,

Thanks a lot for your help! I just wanted to confirm that my problem **was**
 YARN-4452 <https://issues.apache.org/jira/browse/YARN-4452> and it had
been resolved after upgrade to Hadoop 2.7.3

Thank you,
Sergiy.


On Tue, Jan 24, 2017 at 2:59 PM, Jason Lowe <jl...@yahoo-inc.com> wrote:

> Version 2.7.1.2.3.0.0-2557 is not a standard Apache Hadoop version, nor is
> the specified git hash present in the Apache Hadoop repo.  I cannot match
> the code exactly with what you are running.
>
> What is happening according to the RM log is this sequence:
> - The RM is trying to publish system metrics corresponding to the AM
> registration and that blows up with a NullPointerException
> - Because of that NPE the app attempt state machine never properly
> transitions out of the the LAUNCHED state
> - When the AM tries to unregister it is an error because the app is still
> in the LAUNCHED state (due to the incomplete registration)
>
> This looks like the same problem described in YARN-4452
> <https://issues.apache.org/jira/browse/YARN-4452>.  The version you are
> reporting implies it is based on Apache Hadoop 2.7.1, and that problem was
> fixed in 2.7.3.  Check with your vendor and see if that fix is included in
> your version.  I am guessing that it is not and adding that patch will fix
> this use case for you.
>
> Jason
>
>
>
> On Tuesday, January 24, 2017 1:53 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
>
>
> Hi Jason,
>
> Here's what I see in our YARN logs (below). Could it be that I forget to
> populate some field in the registration or, say, application/container
> launch context?
>
> P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
> 9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
> 72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.
>
> Thanks a lot for your help!
>
> Cheers,
> Sergiy.
>
>
> 2017-01-23 19:39:40,544 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
> new applicationId: 1252
> 2017-01-23 19:39:40,575 WARN
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The
> specific
> max attempts: 0 for application: 1252 is invalid, because it is out of the
> range [1, 20]. Use the global max attempts instead.
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
> with id 1252 submitted by user sergiym
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
> application with id application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Submit Application Request
> TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW to NEW_SAVING
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
> Storing info for app: application_1465994698013_1252
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.ParentQueue:
> Application added - appId: application_1465994698013_1252 user: sergiym
> leaf-queue of parent: root #applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Accepted application application_1465994698013_1252 from user: sergiym, in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
> Registering app attempt : appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application application_1465994698013_1252 from user: sergiym activated in
> queue: default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue:
> Application added - appId: application_1465994698013_1252 user:
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.LeafQueue$User@50ac15c8,
> leaf-queue: default #user-pending-applications: 0
> #user-active-applications: 1 #queue-pending-applications: 0
> #queue-active-applications: 1
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.scheduler.
> capacity.CapacityScheduler:
> Added Application Attempt appattempt_1465994698013_1252_000001 to
> scheduler
> from user sergiym in queue default
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Storing attempt: AppId: application_1465994698013_1252 AttemptId:
> appattempt_1465994698013_1252_000001 MasterContainer: null
> 2017-01-23 19:39:40,575 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from SUBMITTED to
> LAUNCHED_UNMANAGED_SAVING
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Create AMRMToken for ApplicationAttempt:
> appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.security.
> AMRMTokenSecretManager:
> Creating password for appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:40,591 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> appattempt_1465994698013_1252_000001 State change from
> LAUNCHED_UNMANAGED_SAVING to LAUNCHED
> 2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
> Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
> registration appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
> IP=10.130.68.120 OPERATION=Register App Master
> TARGET=ApplicationMasterService RESULT=SUCCESS
> APPID=application_1465994698013_1252
> APPATTEMPTID=appattempt_1465994698013_1252_000001
> 2017-01-23 19:39:42,266 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
> handling event type REGISTERED for applicationAttempt
> application_1465994698013_1252
> java.lang.NullPointerException
> at
> org.apache.hadoop.yarn.server.resourcemanager.metrics.
> SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:
> 145)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1405)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.
> java:1380)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.
> doTransition(StateMachineFactory.java:362)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:302)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,266 INFO
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
> application_1465994698013_1252 State change from ACCEPTED to RUNNING
> 2017-01-23 19:39:42,344 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> STATUS_UPDATE at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,407 ERROR
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl:
> Can't handle this event at current state
> org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid
> event:
> UNREGISTERED at LAUNCHED
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(
> StateMachineFactory.java:305)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory.access$
> 300(StateMachineFactory.java:46)
> at
> org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.
> doTransition(StateMachineFactory.java:448)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
> at
> org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.
> RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
> at
> org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$
> ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(
> AsyncDispatcher.java:175)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(
> AsyncDispatcher.java:108)
> at java.lang.Thread.run(Thread.java:745)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8032: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
> for port 8030: readAndProcess from client 10.130.68.120 threw exception
> [java.io.IOException: An existing connection was forcibly closed by the
> remote host]
> java.io.IOException: An existing connection was forcibly closed by the
> remote host
> at sun.nio.ch.SocketDispatcher.read0(Native Method)
> at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
> at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
> at sun.nio.ch.IOUtil.read(IOUtil.java:197)
> at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
> at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
> at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
> at org.apache.hadoop.ipc.Server$Connection.readAndProcess(
> Server.java:1492)
> at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
> at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
> at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
> 2017-01-23 19:50:59,224 INFO
> org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
> Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs
>
>
> On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
> >
> > Have you looked at the ResourceManager logs to see what it is doing when
> it receives the unregister request?  I'm wondering if there's an exception
> or error that could explain why it is not working as expected.  The sample
> code works against trunk for me.  The unmanaged AM registered,
> unregistered, and the final status of the application on the RM was
> FINISHED/SUCCEEDED.
> >
> > Jason
> >
> >
> > On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
> >
> >
> > Hi fellow YARN developers,
> >
> > I am writing a YARN application that runs both Client *and* Unmanaged
> > Application Master in the same JVM process. I have a toy example that
> > starts a YARN application and the AM in Unmanaged mode, and then just
> shuts
> > it down:
> >
> >
> https://github.com/apache/reef/blob/master/lang/java/
> reef-runtime-yarn/src/test/java/org/apache/reef/runtime/
> yarn/driver/unmanaged/UnmanagedAmTest.java
> >
> > (I wrapped it as a unit test, but the code is 100% independent of REEF,
> so
> > you can copy & paste it if you want to play with it; I can also build a
> > small maven project around it).
> >
> > The app *almost* works - the problem seems to be that the call on line
> 117
> >
> >    rmClient.unregisterApplicationMaster(FinalApplicationStatus.
> SUCCEEDED,
> > "Success!", null);
> >
> > fails to update the status of the application on the Resource Manager
> side.
> > I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> > RUNNING/UNDEFINED well after the client/AM process terminates, and
> > eventually got marked as FAILED/FAILED by the RM.
> >
> > Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> > issue that seems to be related to the problem:
> >
> > https://issues.apache.org/jira/browse/YARN-273
> > "Add an unmanaged AM client for in-process AMs"
> >
> > Can someone confirm if my problem is indeed related to that issue, or is
> > there something wrong with my code?
> >
> > Thank you!
> > Sergiy.
> >
> >
>
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Jason Lowe <jl...@yahoo-inc.com.INVALID>.
Version 2.7.1.2.3.0.0-2557 is not a standard Apache Hadoop version, nor is the specified git hash present in the Apache Hadoop repo.  I cannot match the code exactly with what you are running.
What is happening according to the RM log is this sequence:- The RM is trying to publish system metrics corresponding to the AM registration and that blows up with a NullPointerException- Because of that NPE the app attempt state machine never properly transitions out of the the LAUNCHED state- When the AM tries to unregister it is an error because the app is still in the LAUNCHED state (due to the incomplete registration)
This looks like the same problem described in YARN-4452.  The version you are reporting implies it is based on Apache Hadoop 2.7.1, and that problem was fixed in 2.7.3.  Check with your vendor and see if that fix is included in your version.  I am guessing that it is not and adding that patch will fix this use case for you.
Jason

 

    On Tuesday, January 24, 2017 1:53 PM, Sergiy Matusevych <se...@gmail.com> wrote:
 

 Hi Jason,

Here's what I see in our YARN logs (below). Could it be that I forget to
populate some field in the registration or, say, application/container
launch context?

P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.

Thanks a lot for your help!

Cheers,
Sergiy.


2017-01-23 19:39:40,544 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
new applicationId: 1252
2017-01-23 19:39:40,575 WARN
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The specific
max attempts: 0 for application: 1252 is invalid, because it is out of the
range [1, 20]. Use the global max attempts instead.
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
with id 1252 submitted by user sergiym
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
application with id application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Submit Application Request
TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW to NEW_SAVING
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
Storing info for app: application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue:
Application added - appId: application_1465994698013_1252 user: sergiym
leaf-queue of parent: root #applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Accepted application application_1465994698013_1252 from user: sergiym, in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
Registering app attempt : appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application application_1465994698013_1252 from user: sergiym activated in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application added - appId: application_1465994698013_1252 user:
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue$User@50ac15c8,
leaf-queue: default #user-pending-applications: 0
#user-active-applications: 1 #queue-pending-applications: 0
#queue-active-applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Added Application Attempt appattempt_1465994698013_1252_000001 to scheduler
from user sergiym in queue default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Storing attempt: AppId: application_1465994698013_1252 AttemptId:
appattempt_1465994698013_1252_000001 MasterContainer: null
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from SUBMITTED to
LAUNCHED_UNMANAGED_SAVING
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Create AMRMToken for ApplicationAttempt:
appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Creating password for appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from
LAUNCHED_UNMANAGED_SAVING to LAUNCHED
2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
registration appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Register App Master
TARGET=ApplicationMasterService RESULT=SUCCESS
APPID=application_1465994698013_1252
APPATTEMPTID=appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 ERROR
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
handling event type REGISTERED for applicationAttempt
application_1465994698013_1252
java.lang.NullPointerException
at
org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:145)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1405)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1380)
at
org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from ACCEPTED to RUNNING
2017-01-23 19:39:42,344 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
STATUS_UPDATE at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,407 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
UNREGISTERED at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8032: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8030: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:50:59,224 INFO
org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs


On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
>
> Have you looked at the ResourceManager logs to see what it is doing when
it receives the unregister request?  I'm wondering if there's an exception
or error that could explain why it is not working as expected.  The sample
code works against trunk for me.  The unmanaged AM registered,
unregistered, and the final status of the application on the RM was
FINISHED/SUCCEEDED.
>
> Jason
>
>
> On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
sergiy.matusevych@gmail.com> wrote:
>
>
> Hi fellow YARN developers,
>
> I am writing a YARN application that runs both Client *and* Unmanaged
> Application Master in the same JVM process. I have a toy example that
> starts a YARN application and the AM in Unmanaged mode, and then just
shuts
> it down:
>
>
https://github.com/apache/reef/blob/master/lang/java/reef-runtime-yarn/src/test/java/org/apache/reef/runtime/yarn/driver/unmanaged/UnmanagedAmTest.java
>
> (I wrapped it as a unit test, but the code is 100% independent of REEF, so
> you can copy & paste it if you want to play with it; I can also build a
> small maven project around it).
>
> The app *almost* works - the problem seems to be that the call on line 117
>
>    rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
> "Success!", null);
>
> fails to update the status of the application on the Resource Manager
side.
> I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> RUNNING/UNDEFINED well after the client/AM process terminates, and
> eventually got marked as FAILED/FAILED by the RM.
>
> Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> issue that seems to be related to the problem:
>
> https://issues.apache.org/jira/browse/YARN-273
> "Add an unmanaged AM client for in-process AMs"
>
> Can someone confirm if my problem is indeed related to that issue, or is
> there something wrong with my code?
>
> Thank you!
> Sergiy.
>
>


   

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Jason Lowe <jl...@yahoo-inc.com.INVALID>.
Version 2.7.1.2.3.0.0-2557 is not a standard Apache Hadoop version, nor is the specified git hash present in the Apache Hadoop repo.  I cannot match the code exactly with what you are running.
What is happening according to the RM log is this sequence:- The RM is trying to publish system metrics corresponding to the AM registration and that blows up with a NullPointerException- Because of that NPE the app attempt state machine never properly transitions out of the the LAUNCHED state- When the AM tries to unregister it is an error because the app is still in the LAUNCHED state (due to the incomplete registration)
This looks like the same problem described in YARN-4452.  The version you are reporting implies it is based on Apache Hadoop 2.7.1, and that problem was fixed in 2.7.3.  Check with your vendor and see if that fix is included in your version.  I am guessing that it is not and adding that patch will fix this use case for you.
Jason

 

    On Tuesday, January 24, 2017 1:53 PM, Sergiy Matusevych <se...@gmail.com> wrote:
 

 Hi Jason,

Here's what I see in our YARN logs (below). Could it be that I forget to
populate some field in the registration or, say, application/container
launch context?

P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.

Thanks a lot for your help!

Cheers,
Sergiy.


2017-01-23 19:39:40,544 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
new applicationId: 1252
2017-01-23 19:39:40,575 WARN
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The specific
max attempts: 0 for application: 1252 is invalid, because it is out of the
range [1, 20]. Use the global max attempts instead.
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
with id 1252 submitted by user sergiym
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
application with id application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Submit Application Request
TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW to NEW_SAVING
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
Storing info for app: application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue:
Application added - appId: application_1465994698013_1252 user: sergiym
leaf-queue of parent: root #applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Accepted application application_1465994698013_1252 from user: sergiym, in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
Registering app attempt : appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application application_1465994698013_1252 from user: sergiym activated in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application added - appId: application_1465994698013_1252 user:
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue$User@50ac15c8,
leaf-queue: default #user-pending-applications: 0
#user-active-applications: 1 #queue-pending-applications: 0
#queue-active-applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Added Application Attempt appattempt_1465994698013_1252_000001 to scheduler
from user sergiym in queue default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Storing attempt: AppId: application_1465994698013_1252 AttemptId:
appattempt_1465994698013_1252_000001 MasterContainer: null
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from SUBMITTED to
LAUNCHED_UNMANAGED_SAVING
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Create AMRMToken for ApplicationAttempt:
appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Creating password for appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from
LAUNCHED_UNMANAGED_SAVING to LAUNCHED
2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
registration appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Register App Master
TARGET=ApplicationMasterService RESULT=SUCCESS
APPID=application_1465994698013_1252
APPATTEMPTID=appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 ERROR
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
handling event type REGISTERED for applicationAttempt
application_1465994698013_1252
java.lang.NullPointerException
at
org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:145)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1405)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1380)
at
org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from ACCEPTED to RUNNING
2017-01-23 19:39:42,344 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
STATUS_UPDATE at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,407 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
UNREGISTERED at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8032: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8030: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:50:59,224 INFO
org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs


On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
>
> Have you looked at the ResourceManager logs to see what it is doing when
it receives the unregister request?  I'm wondering if there's an exception
or error that could explain why it is not working as expected.  The sample
code works against trunk for me.  The unmanaged AM registered,
unregistered, and the final status of the application on the RM was
FINISHED/SUCCEEDED.
>
> Jason
>
>
> On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
sergiy.matusevych@gmail.com> wrote:
>
>
> Hi fellow YARN developers,
>
> I am writing a YARN application that runs both Client *and* Unmanaged
> Application Master in the same JVM process. I have a toy example that
> starts a YARN application and the AM in Unmanaged mode, and then just
shuts
> it down:
>
>
https://github.com/apache/reef/blob/master/lang/java/reef-runtime-yarn/src/test/java/org/apache/reef/runtime/yarn/driver/unmanaged/UnmanagedAmTest.java
>
> (I wrapped it as a unit test, but the code is 100% independent of REEF, so
> you can copy & paste it if you want to play with it; I can also build a
> small maven project around it).
>
> The app *almost* works - the problem seems to be that the call on line 117
>
>    rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
> "Success!", null);
>
> fails to update the status of the application on the Resource Manager
side.
> I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> RUNNING/UNDEFINED well after the client/AM process terminates, and
> eventually got marked as FAILED/FAILED by the RM.
>
> Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> issue that seems to be related to the problem:
>
> https://issues.apache.org/jira/browse/YARN-273
> "Add an unmanaged AM client for in-process AMs"
>
> Can someone confirm if my problem is indeed related to that issue, or is
> there something wrong with my code?
>
> Thank you!
> Sergiy.
>
>


   

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Hi Jason,

Here's what I see in our YARN logs (below). Could it be that I forget to
populate some field in the registration or, say, application/container
launch context?

P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.

Thanks a lot for your help!

Cheers,
Sergiy.


2017-01-23 19:39:40,544 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
new applicationId: 1252
2017-01-23 19:39:40,575 WARN
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The specific
max attempts: 0 for application: 1252 is invalid, because it is out of the
range [1, 20]. Use the global max attempts instead.
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
with id 1252 submitted by user sergiym
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
application with id application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Submit Application Request
TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW to NEW_SAVING
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
Storing info for app: application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue:
Application added - appId: application_1465994698013_1252 user: sergiym
leaf-queue of parent: root #applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Accepted application application_1465994698013_1252 from user: sergiym, in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
Registering app attempt : appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application application_1465994698013_1252 from user: sergiym activated in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application added - appId: application_1465994698013_1252 user:
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue$User@50ac15c8,
leaf-queue: default #user-pending-applications: 0
#user-active-applications: 1 #queue-pending-applications: 0
#queue-active-applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Added Application Attempt appattempt_1465994698013_1252_000001 to scheduler
from user sergiym in queue default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Storing attempt: AppId: application_1465994698013_1252 AttemptId:
appattempt_1465994698013_1252_000001 MasterContainer: null
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from SUBMITTED to
LAUNCHED_UNMANAGED_SAVING
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Create AMRMToken for ApplicationAttempt:
appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Creating password for appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from
LAUNCHED_UNMANAGED_SAVING to LAUNCHED
2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
registration appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Register App Master
TARGET=ApplicationMasterService RESULT=SUCCESS
APPID=application_1465994698013_1252
APPATTEMPTID=appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 ERROR
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
handling event type REGISTERED for applicationAttempt
application_1465994698013_1252
java.lang.NullPointerException
at
org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:145)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1405)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1380)
at
org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from ACCEPTED to RUNNING
2017-01-23 19:39:42,344 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
STATUS_UPDATE at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,407 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
UNREGISTERED at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8032: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8030: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:50:59,224 INFO
org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs


On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
>
> Have you looked at the ResourceManager logs to see what it is doing when
it receives the unregister request?  I'm wondering if there's an exception
or error that could explain why it is not working as expected.  The sample
code works against trunk for me.  The unmanaged AM registered,
unregistered, and the final status of the application on the RM was
FINISHED/SUCCEEDED.
>
> Jason
>
>
> On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
sergiy.matusevych@gmail.com> wrote:
>
>
> Hi fellow YARN developers,
>
> I am writing a YARN application that runs both Client *and* Unmanaged
> Application Master in the same JVM process. I have a toy example that
> starts a YARN application and the AM in Unmanaged mode, and then just
shuts
> it down:
>
>
https://github.com/apache/reef/blob/master/lang/java/reef-runtime-yarn/src/test/java/org/apache/reef/runtime/yarn/driver/unmanaged/UnmanagedAmTest.java
>
> (I wrapped it as a unit test, but the code is 100% independent of REEF, so
> you can copy & paste it if you want to play with it; I can also build a
> small maven project around it).
>
> The app *almost* works - the problem seems to be that the call on line 117
>
>     rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
> "Success!", null);
>
> fails to update the status of the application on the Resource Manager
side.
> I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> RUNNING/UNDEFINED well after the client/AM process terminates, and
> eventually got marked as FAILED/FAILED by the RM.
>
> Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> issue that seems to be related to the problem:
>
> https://issues.apache.org/jira/browse/YARN-273
> "Add an unmanaged AM client for in-process AMs"
>
> Can someone confirm if my problem is indeed related to that issue, or is
> there something wrong with my code?
>
> Thank you!
> Sergiy.
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Hi Jason,

Thanks a lot for your help! It is great to hear that Unmanaged AM mode
works at least on some version of YARN :) I will check our YARN logs and
give you more details soon.

Thank you!
Sergiy.

On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:

> Have you looked at the ResourceManager logs to see what it is doing when
> it receives the unregister request?  I'm wondering if there's an exception
> or error that could explain why it is not working as expected.  The sample
> code works against trunk for me.  The unmanaged AM registered,
> unregistered, and the final status of the application on the RM was
> FINISHED/SUCCEEDED.
>
> Jason
>
>
> On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
>
>
> Hi fellow YARN developers,
>
> I am writing a YARN application that runs both Client *and* Unmanaged
> Application Master in the same JVM process. I have a toy example that
> starts a YARN application and the AM in Unmanaged mode, and then just shuts
> it down:
>
> https://github.com/apache/reef/blob/master/lang/java/
> reef-runtime-yarn/src/test/java/org/apache/reef/runtime/
> yarn/driver/unmanaged/UnmanagedAmTest.java
>
> (I wrapped it as a unit test, but the code is 100% independent of REEF, so
> you can copy & paste it if you want to play with it; I can also build a
> small maven project around it).
>
> The app *almost* works - the problem seems to be that the call on line 117
>
>     rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
> "Success!", null);
>
> fails to update the status of the application on the Resource Manager side.
> I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> RUNNING/UNDEFINED well after the client/AM process terminates, and
> eventually got marked as FAILED/FAILED by the RM.
>
> Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> issue that seems to be related to the problem:
>
> https://issues.apache.org/jira/browse/YARN-273
> "Add an unmanaged AM client for in-process AMs"
>
> Can someone confirm if my problem is indeed related to that issue, or is
> there something wrong with my code?
>
> Thank you!
> Sergiy.
>
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Hi Jason,

Thanks a lot for your help! It is great to hear that Unmanaged AM mode
works at least on some version of YARN :) I will check our YARN logs and
give you more details soon.

Thank you!
Sergiy.

On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:

> Have you looked at the ResourceManager logs to see what it is doing when
> it receives the unregister request?  I'm wondering if there's an exception
> or error that could explain why it is not working as expected.  The sample
> code works against trunk for me.  The unmanaged AM registered,
> unregistered, and the final status of the application on the RM was
> FINISHED/SUCCEEDED.
>
> Jason
>
>
> On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
> sergiy.matusevych@gmail.com> wrote:
>
>
> Hi fellow YARN developers,
>
> I am writing a YARN application that runs both Client *and* Unmanaged
> Application Master in the same JVM process. I have a toy example that
> starts a YARN application and the AM in Unmanaged mode, and then just shuts
> it down:
>
> https://github.com/apache/reef/blob/master/lang/java/
> reef-runtime-yarn/src/test/java/org/apache/reef/runtime/
> yarn/driver/unmanaged/UnmanagedAmTest.java
>
> (I wrapped it as a unit test, but the code is 100% independent of REEF, so
> you can copy & paste it if you want to play with it; I can also build a
> small maven project around it).
>
> The app *almost* works - the problem seems to be that the call on line 117
>
>     rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
> "Success!", null);
>
> fails to update the status of the application on the Resource Manager side.
> I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> RUNNING/UNDEFINED well after the client/AM process terminates, and
> eventually got marked as FAILED/FAILED by the RM.
>
> Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> issue that seems to be related to the problem:
>
> https://issues.apache.org/jira/browse/YARN-273
> "Add an unmanaged AM client for in-process AMs"
>
> Can someone confirm if my problem is indeed related to that issue, or is
> there something wrong with my code?
>
> Thank you!
> Sergiy.
>
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Sergiy Matusevych <se...@gmail.com>.
Hi Jason,

Here's what I see in our YARN logs (below). Could it be that I forget to
populate some field in the registration or, say, application/container
launch context?

P.S. YARN version that we are running is: 2.7.1.2.3.0.0-2557 from
9f17d40a0f2046d217b2bff90ad6e2fc7e41f5e1 by jenkins source checksum
72da60cd1e57ff22763939f46330224c on 2015-07-14T13:30Z.

Thanks a lot for your help!

Cheers,
Sergiy.


2017-01-23 19:39:40,544 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Allocated
new applicationId: 1252
2017-01-23 19:39:40,575 WARN
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: The specific
max attempts: 0 for application: 1252 is invalid, because it is out of the
range [1, 20]. Use the global max attempts instead.
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ClientRMService: Application
with id 1252 submitted by user sergiym
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: Storing
application with id application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Submit Application Request
TARGET=ClientRMService RESULT=SUCCESS APPID=application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW to NEW_SAVING
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore:
Storing info for app: application_1465994698013_1252
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from NEW_SAVING to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.ParentQueue:
Application added - appId: application_1465994698013_1252 user: sergiym
leaf-queue of parent: root #applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Accepted application application_1465994698013_1252 from user: sergiym, in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from SUBMITTED to ACCEPTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService:
Registering app attempt : appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from NEW to SUBMITTED
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application application_1465994698013_1252 from user: sergiym activated in
queue: default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue:
Application added - appId: application_1465994698013_1252 user:
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.LeafQueue$User@50ac15c8,
leaf-queue: default #user-pending-applications: 0
#user-active-applications: 1 #queue-pending-applications: 0
#queue-active-applications: 1
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.CapacityScheduler:
Added Application Attempt appattempt_1465994698013_1252_000001 to scheduler
from user sergiym in queue default
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Storing attempt: AppId: application_1465994698013_1252 AttemptId:
appattempt_1465994698013_1252_000001 MasterContainer: null
2017-01-23 19:39:40,575 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from SUBMITTED to
LAUNCHED_UNMANAGED_SAVING
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Create AMRMToken for ApplicationAttempt:
appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager:
Creating password for appattempt_1465994698013_1252_000001
2017-01-23 19:39:40,591 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
appattempt_1465994698013_1252_000001 State change from
LAUNCHED_UNMANAGED_SAVING to LAUNCHED
2017-01-23 19:39:42,250 INFO SecurityLogger.org.apache.hadoop.ipc.Server:
Auth successful for appattempt_1465994698013_1252_000001 (auth:SIMPLE)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService: AM
registration appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger: USER=sergiym
IP=10.130.68.120 OPERATION=Register App Master
TARGET=ApplicationMasterService RESULT=SUCCESS
APPID=application_1465994698013_1252
APPATTEMPTID=appattempt_1465994698013_1252_000001
2017-01-23 19:39:42,266 ERROR
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager: Error in
handling event type REGISTERED for applicationAttempt
application_1465994698013_1252
java.lang.NullPointerException
at
org.apache.hadoop.yarn.server.resourcemanager.metrics.SystemMetricsPublisher.appAttemptRegistered(SystemMetricsPublisher.java:145)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1405)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl$AMRegisteredTransition.transition(RMAppAttemptImpl.java:1380)
at
org.apache.hadoop.yarn.state.StateMachineFactory$SingleInternalArc.doTransition(StateMachineFactory.java:362)
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:302)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,266 INFO
org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl:
application_1465994698013_1252 State change from ACCEPTED to RUNNING
2017-01-23 19:39:42,344 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
STATUS_UPDATE at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,407 ERROR
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl:
Can't handle this event at current state
org.apache.hadoop.yarn.state.InvalidStateTransitonException: Invalid event:
UNREGISTERED at LAUNCHED
at
org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305)
at
org.apache.hadoop.yarn.state.StateMachineFactory.access$300(StateMachineFactory.java:46)
at
org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:448)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:786)
at
org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:108)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:850)
at
org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:831)
at
org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:175)
at
org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:108)
at java.lang.Thread.run(Thread.java:745)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8032: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:39:42,453 INFO org.apache.hadoop.ipc.Server: Socket Reader #1
for port 8030: readAndProcess from client 10.130.68.120 threw exception
[java.io.IOException: An existing connection was forcibly closed by the
remote host]
java.io.IOException: An existing connection was forcibly closed by the
remote host
at sun.nio.ch.SocketDispatcher.read0(Native Method)
at sun.nio.ch.SocketDispatcher.read(SocketDispatcher.java:43)
at sun.nio.ch.IOUtil.readIntoNativeBuffer(IOUtil.java:223)
at sun.nio.ch.IOUtil.read(IOUtil.java:197)
at sun.nio.ch.SocketChannelImpl.read(SocketChannelImpl.java:380)
at org.apache.hadoop.ipc.Server.channelRead(Server.java:2635)
at org.apache.hadoop.ipc.Server.access$2800(Server.java:136)
at org.apache.hadoop.ipc.Server$Connection.readAndProcess(Server.java:1492)
at org.apache.hadoop.ipc.Server$Listener.doRead(Server.java:782)
at org.apache.hadoop.ipc.Server$Listener$Reader.doRunLoop(Server.java:648)
at org.apache.hadoop.ipc.Server$Listener$Reader.run(Server.java:619)
2017-01-23 19:50:59,224 INFO
org.apache.hadoop.yarn.util.AbstractLivelinessMonitor:
Expired:appattempt_1465994698013_1252_000001 Timed out after 600 secs


On Tue, Jan 24, 2017 at 8:48 AM, Jason Lowe <jl...@yahoo-inc.com> wrote:
>
> Have you looked at the ResourceManager logs to see what it is doing when
it receives the unregister request?  I'm wondering if there's an exception
or error that could explain why it is not working as expected.  The sample
code works against trunk for me.  The unmanaged AM registered,
unregistered, and the final status of the application on the RM was
FINISHED/SUCCEEDED.
>
> Jason
>
>
> On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <
sergiy.matusevych@gmail.com> wrote:
>
>
> Hi fellow YARN developers,
>
> I am writing a YARN application that runs both Client *and* Unmanaged
> Application Master in the same JVM process. I have a toy example that
> starts a YARN application and the AM in Unmanaged mode, and then just
shuts
> it down:
>
>
https://github.com/apache/reef/blob/master/lang/java/reef-runtime-yarn/src/test/java/org/apache/reef/runtime/yarn/driver/unmanaged/UnmanagedAmTest.java
>
> (I wrapped it as a unit test, but the code is 100% independent of REEF, so
> you can copy & paste it if you want to play with it; I can also build a
> small maven project around it).
>
> The app *almost* works - the problem seems to be that the call on line 117
>
>     rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
> "Success!", null);
>
> fails to update the status of the application on the Resource Manager
side.
> I would expect it to be FINISHED/SUCCEDED, but instead it stays in
> RUNNING/UNDEFINED well after the client/AM process terminates, and
> eventually got marked as FAILED/FAILED by the RM.
>
> Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
> issue that seems to be related to the problem:
>
> https://issues.apache.org/jira/browse/YARN-273
> "Add an unmanaged AM client for in-process AMs"
>
> Can someone confirm if my problem is indeed related to that issue, or is
> there something wrong with my code?
>
> Thank you!
> Sergiy.
>
>

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Jason Lowe <jl...@yahoo-inc.com.INVALID>.
Have you looked at the ResourceManager logs to see what it is doing when it receives the unregister request?  I'm wondering if there's an exception or error that could explain why it is not working as expected.  The sample code works against trunk for me.  The unmanaged AM registered, unregistered, and the final status of the application on the RM was FINISHED/SUCCEEDED.
Jason
 

    On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <se...@gmail.com> wrote:
 

 Hi fellow YARN developers,

I am writing a YARN application that runs both Client *and* Unmanaged
Application Master in the same JVM process. I have a toy example that
starts a YARN application and the AM in Unmanaged mode, and then just shuts
it down:

https://github.com/apache/reef/blob/master/lang/java/reef-runtime-yarn/src/test/java/org/apache/reef/runtime/yarn/driver/unmanaged/UnmanagedAmTest.java

(I wrapped it as a unit test, but the code is 100% independent of REEF, so
you can copy & paste it if you want to play with it; I can also build a
small maven project around it).

The app *almost* works - the problem seems to be that the call on line 117

    rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
"Success!", null);

fails to update the status of the application on the Resource Manager side.
I would expect it to be FINISHED/SUCCEDED, but instead it stays in
RUNNING/UNDEFINED well after the client/AM process terminates, and
eventually got marked as FAILED/FAILED by the RM.

Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
issue that seems to be related to the problem:

https://issues.apache.org/jira/browse/YARN-273
"Add an unmanaged AM client for in-process AMs"

Can someone confirm if my problem is indeed related to that issue, or is
there something wrong with my code?

Thank you!
Sergiy.


   

Re: YARN Client and Unmanaged AM running in the same process?

Posted by Jason Lowe <jl...@yahoo-inc.com.INVALID>.
Have you looked at the ResourceManager logs to see what it is doing when it receives the unregister request?  I'm wondering if there's an exception or error that could explain why it is not working as expected.  The sample code works against trunk for me.  The unmanaged AM registered, unregistered, and the final status of the application on the RM was FINISHED/SUCCEEDED.
Jason
 

    On Monday, January 23, 2017 9:51 PM, Sergiy Matusevych <se...@gmail.com> wrote:
 

 Hi fellow YARN developers,

I am writing a YARN application that runs both Client *and* Unmanaged
Application Master in the same JVM process. I have a toy example that
starts a YARN application and the AM in Unmanaged mode, and then just shuts
it down:

https://github.com/apache/reef/blob/master/lang/java/reef-runtime-yarn/src/test/java/org/apache/reef/runtime/yarn/driver/unmanaged/UnmanagedAmTest.java

(I wrapped it as a unit test, but the code is 100% independent of REEF, so
you can copy & paste it if you want to play with it; I can also build a
small maven project around it).

The app *almost* works - the problem seems to be that the call on line 117

    rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
"Success!", null);

fails to update the status of the application on the Resource Manager side.
I would expect it to be FINISHED/SUCCEDED, but instead it stays in
RUNNING/UNDEFINED well after the client/AM process terminates, and
eventually got marked as FAILED/FAILED by the RM.

Am I doing something wrong, or is that a bug in YARN? I found an old JIRA
issue that seems to be related to the problem:

https://issues.apache.org/jira/browse/YARN-273
"Add an unmanaged AM client for in-process AMs"

Can someone confirm if my problem is indeed related to that issue, or is
there something wrong with my code?

Thank you!
Sergiy.