You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "ASF GitHub Bot (JIRA)" <ji...@apache.org> on 2018/03/07 03:50:00 UTC

[jira] [Commented] (FLINK-8887) ClusterClient.getJobStatus can throw FencingTokenException

    [ https://issues.apache.org/jira/browse/FLINK-8887?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16388967#comment-16388967 ] 

ASF GitHub Bot commented on FLINK-8887:
---------------------------------------

GitHub user yanghua opened a pull request:

    https://github.com/apache/flink/pull/5648

    [FLINK-8887][flip-6] ClusterClient.getJobStatus can throw FencingTokenException

    ## What is the purpose of the change
    
    *This pull request fixed ClusterClient.getJobStatus throw FencingTokenException issue*
    
    
    ## Brief change log
    
      - *try-catch request job status from job master gateway if catch a exception then goto : request job status  from archived execution graph store*
    
    ## Verifying this change
    
    This change is already covered by existing tests, such as *DispatcherTest.testCacheJobExecutionResult*.
    
    ## Does this pull request potentially affect one of the following parts:
    
      - Dependencies (does it add or upgrade a dependency): (yes / **no**)
      - The public API, i.e., is any changed class annotated with `@Public(Evolving)`: (yes / **no**)
      - The serializers: (yes / **no** / don't know)
      - The runtime per-record code paths (performance sensitive): (yes / **no** / don't know)
      - Anything that affects deployment or recovery: JobManager (and its components), Checkpointing, Yarn/Mesos, ZooKeeper: (yes / **no** / don't know)
      - The S3 file system connector: (yes / **no** / don't know)
    
    ## Documentation
    
      - Does this pull request introduce a new feature? (yes / **no**)
      - If yes, how is the feature documented? (not applicable / docs / JavaDocs / **not documented**)


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/yanghua/flink FLINK-8887

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/flink/pull/5648.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #5648
    
----
commit 03c5bf0aa1536842359740b53f43e7b46f48b006
Author: vinoyang <vi...@...>
Date:   2018-03-07T03:43:10Z

    [FLINK-8887][flip-6] ClusterClient.getJobStatus can throw FencingTokenException

----


> ClusterClient.getJobStatus can throw FencingTokenException
> ----------------------------------------------------------
>
>                 Key: FLINK-8887
>                 URL: https://issues.apache.org/jira/browse/FLINK-8887
>             Project: Flink
>          Issue Type: Bug
>          Components: Distributed Coordination
>    Affects Versions: 1.5.0
>            Reporter: Gary Yao
>            Assignee: vinoyang
>            Priority: Blocker
>              Labels: flip-6
>             Fix For: 1.5.0
>
>
> *Description*
> Calling {{RestClusterClient.getJobStatus}} or {{MiniClusterClient.getJobStatus}} can result in a {{FencingTokenException}}. 
> *Analysis*
> {{Dispatcher.requestJobStatus}} first looks the {{JobManagerRunner}} up by job id. If a reference is found, {{requestJobStatus}} is called on the respective instance. If not, the {{ArchivedExecutionGraphStore}} is queried. However, between the lookup and the method call, the {{JobMaster}} of the respective job may have lost leadership already (job finished), and has set the fencing token to {{null}}.
> *Stacktrace*
> {noformat}
> Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token mismatch: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestJobStatus(Time))) because the fencing token null did not match the expected fencing token b8423c75bc6838244b8c93c8bd4a4f51.
> 	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleMessage(FencedAkkaRpcActor.java:73)
> 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$onReceive$1(AkkaRpcActor.java:132)
> 	at akka.actor.ActorCell$$anonfun$become$1.applyOrElse(ActorCell.scala:544)
> 	at akka.actor.Actor$class.aroundReceive(Actor.scala:502)
> 	at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)
> 	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)
> 	at akka.actor.ActorCell.invoke(ActorCell.scala:495)
> 	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)
> 	at akka.dispatch.Mailbox.run(Mailbox.scala:224)
> 	at akka.dispatch.Mailbox.exec(Mailbox.scala:234)
> 	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> 	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> 	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> 	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> {noformat}
> {noformat}
> Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestJobStatus(Time))) because the fencing token is null.
> 	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleMessage(FencedAkkaRpcActor.java:56)
> 	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.lambda$onReceive$1(AkkaRpcActor.java:132)
> 	at akka.actor.ActorCell$$anonfun$become$1.applyOrElse(ActorCell.scala:544)
> 	at akka.actor.Actor$class.aroundReceive(Actor.scala:502)
> 	at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)
> 	at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)
> 	at akka.actor.ActorCell.invoke(ActorCell.scala:495)
> 	at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)
> 	at akka.dispatch.Mailbox.run(Mailbox.scala:224)
> 	at akka.dispatch.Mailbox.exec(Mailbox.scala:234)
> 	at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> 	at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> 	at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> 	at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> {noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)