[ https://issues.apache.org/jira/browse/YARN-10232?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
YCozy updated YARN-10232: ------------------------- Description: We were testing YARN under network partition and found the following ERROR in RM's log. {code:java} 2020-04-11 13:10:39,739 ERROR org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: App attempt: appattempt_6_0001_000002 can't handle this event at current state org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: LAUNCH_FAILED at RUNNING at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305) at org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46) at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:916) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:121) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1097) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1078) at org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:222) at org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:138) at java.lang.Thread.run(Thread.java:748) {code} After analyzing the logs, we have recovered the triggering process of this bug: * We have a cluster with one RM and one NM. * A client tries to start a YARN service. * RM send a request to NM to start the containers: NM's log: {code:java} 2020-04-11 14:23:44,030 INFO SecurityLogger.org.apache.hadoop.ipc.Server: Auth successful for appattempt_6_0001_000002 (auth:SIMPLE) 2020-04-11 14:23:44,229 INFO org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl: Start request for container_6_0001_02_000001 by user appattempt_6_0001_000002 {code} * NM starts the containers successfully: NM's log: {code:java} 2020-04-11 14:23:44,347 INFO org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl: Application application_6_0001 transitioned from INITING to RUNNING 2020-04-11 14:23:44,357 INFO org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl: Container container_6_0001_02_000001 transitioned from NEW to LOCALIZING {code} * However, due to network partition, NM failed to send back the RPC response. * After a while, the application is running happily: RM's log: {code:java} 2020-04-11 14:23:50,359 INFO org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: appattempt_6_0001_000002 State change from ALLOCATED to RUNNING on event = REGISTERED 020-04-11 14:23:50,359 INFO org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: application_6_0001 State change from ACCEPTED to RUNNING on event = ATTEMPT_REGISTERED{code} * Then, since RM didn't receive the RPC response for startContainers, it retries. The network partition has already stopped, so NM will receive the new startContainers RPC: NM's log: {code:java} 2020-04-11 14:23:54,392 INFO SecurityLogger.org.apache.hadoop.ipc.Server: Auth successful for appattempt_6_0001_000002 (auth:SIMPLE) {code} * But since the attempt is actually running, this launch request does not succeed: NM's log: {code:java} 2020-04-11 14:23:54,401 ERROR org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl: Unauthorized request to start container. Attempt to relaunch the same container with id container_6_0001_02_000001. {code} RM's log: {code:java} 2020-04-11 14:23:54,428 INFO org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher: Error launching appattempt_6_0001_000002. Got exception: org.apache.hadoop.yarn.exceptions.YarnException: Unauthorized request to start container. Attempt to relaunch the same container with id container_6_0001_02_000001. at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.instantiateExceptionImpl(SerializedExceptionPBImpl.java:171) at org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.instantiateException(SerializedExceptionPBImpl.java:182) at org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.deSerialize(SerializedExceptionPBImpl.java:106) at org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.launch(AMLauncher.java:137) at org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.run(AMLauncher.java:344) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748){code} * As a result, RM gets the following InvalidStateTransitionException ERROR: RM's log: {code:java} 2020-04-11 14:23:54,429 ERROR org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: App attempt: appattempt_6_0001_000002 can't handle this event at current state org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: LAUNCH_FAILED at RUNNING at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305) at org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46) at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:916) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:121) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1097) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1078) at org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:222) at org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:138) at java.lang.Thread.run(Thread.java:748) {code} was: We were testing YARN under network partition and found the following ERROR in RM's log. {code:java} 2020-04-11 13:10:39,739 ERROR org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: App attempt: appattempt_6_0001_000002 can't handle this event at current state org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: LAUNCH_FAILED at RUNNING at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305) at org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46) at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:916) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:121) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1097) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1078) at org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:222) at org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:138) at java.lang.Thread.run(Thread.java:748) {code} After analyzing the logs, we have recovered the triggering process of this bug: * We have a cluster with one RM and one NM. * A client tries to start a YARN service. * RM send a request to NM to start the containers: NM's log: {code:java} 2020-04-11 14:23:44,030 INFO SecurityLogger.org.apache.hadoop.ipc.Server: Auth successful for appattempt_6_0001_000002 (auth:SIMPLE) 2020-04-11 14:23:44,229 INFO org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl: Start request for container_6_0001_02_000001 by user appattempt_6_0001_000002 {code} * NM starts the containers successfully: NM's log: {code:java} 2020-04-11 14:23:44,347 INFO org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl: Application application_6_0001 transitioned from INITING to RUNNING 2020-04-11 14:23:44,357 INFO org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl: Container container_6_0001_02_000001 transitioned from NEW to LOCALIZING {code} * However, due to network partition, NM failed to send back the RPC response. * After a while, the application is running happily: RM's log: {code:java} 2020-04-11 14:23:50,359 INFO org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: appattempt_6_0001_000002 State change from ALLOCATED to RUNNING on event = REGISTERED 020-04-11 14:23:50,359 INFO org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: application_6_0001 State change from ACCEPTED to RUNNING on event = ATTEMPT_REGISTERED{code} * Then, since RM didn't receive the RPC response for startContainers, it retries: NM's log: {code:java} 2020-04-11 14:23:54,392 INFO SecurityLogger.org.apache.hadoop.ipc.Server: Auth successful for appattempt_6_0001_000002 (auth:SIMPLE) {code} * But since the attempt is actually running, this launch request does not succeed: NM's log: {code:java} 2020-04-11 14:23:54,401 ERROR org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl: Unauthorized request to start container. Attempt to relaunch the same container with id container_6_0001_02_000001. {code} RM's log: {code:java} 2020-04-11 14:23:54,428 INFO org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher: Error launching appattempt_6_0001_000002. Got exception: org.apache.hadoop.yarn.exceptions.YarnException: Unauthorized request to start container. Attempt to relaunch the same container with id container_6_0001_02_000001. at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) at sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) at sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) at java.lang.reflect.Constructor.newInstance(Constructor.java:423) at org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.instantiateExceptionImpl(SerializedExceptionPBImpl.java:171) at org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.instantiateException(SerializedExceptionPBImpl.java:182) at org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.deSerialize(SerializedExceptionPBImpl.java:106) at org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.launch(AMLauncher.java:137) at org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.run(AMLauncher.java:344) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748){code} * As a result, RM gets the following InvalidStateTransitionException ERROR: RM's log: {code:java} 2020-04-11 14:23:54,429 ERROR org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: App attempt: appattempt_6_0001_000002 can't handle this event at current state org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: LAUNCH_FAILED at RUNNING at org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305) at org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46) at org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:916) at org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:121) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1097) at org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1078) at org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:222) at org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:138) at java.lang.Thread.run(Thread.java:748) {code} > InvalidStateTransitionException: Invalid event: LAUNCH_FAILED at RUNNING > ------------------------------------------------------------------------ > > Key: YARN-10232 > URL: https://issues.apache.org/jira/browse/YARN-10232 > Project: Hadoop YARN > Issue Type: Bug > Reporter: YCozy > Priority: Major > > We were testing YARN under network partition and found the following ERROR in > RM's log. > {code:java} > 2020-04-11 13:10:39,739 ERROR > org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: > App attempt: appattempt_6_0001_000002 can't handle this event at current state > org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: > LAUNCH_FAILED at RUNNING > at > org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305) > at > org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46) > at > org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487) > at > org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:916) > at > org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:121) > at > org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1097) > at > org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1078) > at > org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:222) > at > org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:138) > at java.lang.Thread.run(Thread.java:748) > {code} > After analyzing the logs, we have recovered the triggering process of this > bug: > * We have a cluster with one RM and one NM. > * A client tries to start a YARN service. > * RM send a request to NM to start the containers: > NM's log: > {code:java} > 2020-04-11 14:23:44,030 INFO SecurityLogger.org.apache.hadoop.ipc.Server: > Auth successful for appattempt_6_0001_000002 (auth:SIMPLE) > 2020-04-11 14:23:44,229 INFO > org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl: > Start request for container_6_0001_02_000001 by user appattempt_6_0001_000002 > {code} > * NM starts the containers successfully: > NM's log: > {code:java} > 2020-04-11 14:23:44,347 INFO > org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationImpl: > Application application_6_0001 transitioned from INITING to RUNNING > 2020-04-11 14:23:44,357 INFO > org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerImpl: > Container container_6_0001_02_000001 transitioned from NEW to LOCALIZING > {code} > * However, due to network partition, NM failed to send back the RPC response. > * After a while, the application is running happily: > RM's log: > {code:java} > 2020-04-11 14:23:50,359 INFO > org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: > appattempt_6_0001_000002 State change from ALLOCATED to RUNNING on event = > REGISTERED > 020-04-11 14:23:50,359 INFO > org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppImpl: > application_6_0001 State change from ACCEPTED to RUNNING on event = > ATTEMPT_REGISTERED{code} > * Then, since RM didn't receive the RPC response for startContainers, it > retries. The network partition has already stopped, so NM will receive the > new startContainers RPC: > NM's log: > {code:java} > 2020-04-11 14:23:54,392 INFO SecurityLogger.org.apache.hadoop.ipc.Server: > Auth successful for appattempt_6_0001_000002 (auth:SIMPLE) > {code} > * But since the attempt is actually running, this launch request does not > succeed: > NM's log: > {code:java} > 2020-04-11 14:23:54,401 ERROR > org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl: > Unauthorized request to start container. > Attempt to relaunch the same container with id container_6_0001_02_000001. > {code} > RM's log: > {code:java} > 2020-04-11 14:23:54,428 INFO > org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher: Error > launching appattempt_6_0001_000002. Got exception: > org.apache.hadoop.yarn.exceptions.YarnException: Unauthorized request to > start container. > Attempt to relaunch the same container with id container_6_0001_02_000001. > at sun.reflect.NativeConstructorAccessorImpl.newInstance0(Native Method) > at > sun.reflect.NativeConstructorAccessorImpl.newInstance(NativeConstructorAccessorImpl.java:62) > at > sun.reflect.DelegatingConstructorAccessorImpl.newInstance(DelegatingConstructorAccessorImpl.java:45) > at java.lang.reflect.Constructor.newInstance(Constructor.java:423) > at > org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.instantiateExceptionImpl(SerializedExceptionPBImpl.java:171) > at > org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.instantiateException(SerializedExceptionPBImpl.java:182) > at > org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl.deSerialize(SerializedExceptionPBImpl.java:106) > at > org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.launch(AMLauncher.java:137) > at > org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncher.run(AMLauncher.java:344) > at > java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) > at > java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) > at java.lang.Thread.run(Thread.java:748){code} > * As a result, RM gets the following InvalidStateTransitionException ERROR: > RM's log: > {code:java} > 2020-04-11 14:23:54,429 ERROR > org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl: > App attempt: appattempt_6_0001_000002 can't handle this event at current state > org.apache.hadoop.yarn.state.InvalidStateTransitionException: Invalid event: > LAUNCH_FAILED at RUNNING > at > org.apache.hadoop.yarn.state.StateMachineFactory.doTransition(StateMachineFactory.java:305) > at > org.apache.hadoop.yarn.state.StateMachineFactory.access$500(StateMachineFactory.java:46) > at > org.apache.hadoop.yarn.state.StateMachineFactory$InternalStateMachine.doTransition(StateMachineFactory.java:487) > at > org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:916) > at > org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptImpl.handle(RMAppAttemptImpl.java:121) > at > org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1097) > at > org.apache.hadoop.yarn.server.resourcemanager.ResourceManager$ApplicationAttemptEventDispatcher.handle(ResourceManager.java:1078) > at > org.apache.hadoop.yarn.event.AsyncDispatcher.dispatch(AsyncDispatcher.java:222) > at > org.apache.hadoop.yarn.event.AsyncDispatcher$1.run(AsyncDispatcher.java:138) > at java.lang.Thread.run(Thread.java:748) > {code} > -- This message was sent by Atlassian Jira (v8.3.4#803005) --------------------------------------------------------------------- To unsubscribe, e-mail: yarn-issues-unsubscr...@hadoop.apache.org For additional commands, e-mail: yarn-issues-h...@hadoop.apache.org