Fencing token exceptions from Job Manager High Availability mode

classic Classic list List threaded Threaded
10 messages Options
Reply | Threaded
Open this post in threaded view
|

Fencing token exceptions from Job Manager High Availability mode

Hanson, Bruce

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 

Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Fabian Hueske-2
Hi Bruce,

I haven't seen such an exception yet, but maybe Till (in CC) can help.

Best,
Fabian

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 

Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Till Rohrmann
Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

Cheers,
Till

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:
Hi Bruce,

I haven't seen such an exception yet, but maybe Till (in CC) can help.

Best,
Fabian

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 

Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Joshua Fan
Hi Till

Recently, this issue happened frequently, but some jobs recovered from the exception, some not.

for the job who can not recover from the exception, the log pasted below:

2019-10-09 13:54:21.746 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: SUSPENDED
2019-10-09 13:54:21.746 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.162.89.153:16736/user/dispatcher no longer participates in the leader election.
2019-10-09 13:54:21.746 [flink-akka.actor.default-dispatcher-46] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.162.89.153:16736/user/resourcemanager was revoked leadership. Clearing fencing token.
2019-10-09 13:54:21.746 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender http://10.162.89.153:12503 no longer participates in the leader election.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] INFO  o.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint  - http://10.162.89.153:12503 lost leadership
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.162.89.153:16736/user/resourcemanager no longer participates in the leader election.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.747 [flink-akka.actor.default-dispatcher-43] INFO  org.apache.flink.runtime.dispatcher.MiniDispatcher  - Dispatcher akka.tcp://flink@10.162.89.153:16736/user/dispatcher was revoked leadership.
2019-10-09 13:54:21.747 [flink-akka.actor.default-dispatcher-46] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Suspending the SlotManager.
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.26/10.160.101.26:2181
2019-10-09">10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:22.072 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:22.973 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.973 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.27/10.160.101.27:2181
2019-10-09">10.160.101.27/10.160.101.27:2181
2019-10-09 13:54:22.973 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.974 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:23.405 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:23.405 [main-SendThread(10.160.101.30:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.30/10.160.101.30:2181
2019-10-09">10.160.101.30/10.160.101.30:2181
2019-10-09 13:54:23.405 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:23.406 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.379 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.379 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.28/10.160.101.28:2181
2019-10-09">10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:24.379 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.380 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:26.044 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:26.045 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.29/10.160.101.29:2181
2019-10-09">10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:26.045 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:29.050 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.29/10.160.101.29:2181, initiating session
2019-10-09 13:54:29.095 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session establishment complete on server 10.160.101.29/10.160.101.29:2181, sessionid = 0x2597758ae5091ff, negotiated timeout = 40000
2019-10-09 13:54:29.095 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: RECONNECTED
2019-10-09 13:54:29.096 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:29.105 [flink-akka.actor.default-dispatcher-46] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.162.89.153:16736/user/resourcemanager was granted leadership with fencing token a22e4c9c80fd0c7249102f69c0b64e01
2019-10-09 13:54:29.105 [flink-akka.actor.default-dispatcher-46] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Starting the SlotManager.
2019-10-09 13:54:29.107 [main-EventThread] INFO  o.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint  - http://10.162.89.153:12503 was granted leadership with leaderSessionID=53b02e00-95c3-4d34-a272-b84795b991a7
2019-10-09 13:54:29.116 [flink-akka.actor.default-dispatcher-26] INFO  org.apache.flink.runtime.dispatcher.MiniDispatcher  - Dispatcher akka.tcp://flink@10.162.89.153:16736/user/dispatcher was granted leadership with fencing token d645a3fe-8fee-453b-abd5-bd4cd82ebba1
2019-10-09 13:54:34.809 [flink-rest-server-netty-worker-thread-73] ERROR org.apache.flink.runtime.rest.handler.job.JobDetailsHandler  - Could not retrieve the redirect address.
java.util.concurrent.CompletionException: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:772)
	at akka.dispatch.OnComplete.internal(Future.scala:258)
	at akka.dispatch.OnComplete.internal(Future.scala:256)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
	at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
	at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
	at akka.actor.ActorRef.tell(ActorRef.scala:130)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendErrorIfSender(AkkaRpcActor.java:371)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:57)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)
	at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)
	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)
Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)
	... 14 common frames omitted
2019-10-09 13:54:56.007 [flink-rest-server-netty-worker-thread-74] ERROR org.apache.flink.runtime.rest.handler.job.JobDetailsHandler  - Could not retrieve the redirect address.
java.util.concurrent.CompletionException: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:772)
	at akka.dispatch.OnComplete.internal(Future.scala:258)
	at akka.dispatch.OnComplete.internal(Future.scala:256)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
	at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
	at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
	at akka.actor.ActorRef.tell(ActorRef.scala:130)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendErrorIfSender(AkkaRpcActor.java:371)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:57)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)
	at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)
	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)
Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)
	... 14 common frames omitted

In the above log, the ha zk got reconnected after a suspend, but the job can not recover, and there existed a MiniDispatcherRestEndpoint.


for the job who can recover from the same condition, the log pasted as below:

2019-10-09 13:54:21.803 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: SUSPENDED
2019-10-09 13:54:21.804 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender http://10.160.140.216:52303 no longer participates in the leader election.
2019-10-09 13:54:21.804 [Curator-ConnectionStateManager-0] INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint  - http://10.160.140.216:52303 lost leadership
2019-10-09 13:54:21.804 [Curator-PathChildrenCache-0] WARN  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - ZooKeeper connection SUSPENDING. Changes to the submitted job graphs are not monitored (temporarily).
2019-10-09 13:54:21.805 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.805 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.160.140.216:26073/user/resourcemanager was revoked leadership. Clearing fencing token.
2019-10-09 13:54:21.805 [Curator-ConnectionStateManager-0] INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner  - JobManager for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) was revoked leadership at akka.tcp://<a href="http://flink@10.160.140.216:26073/user/jobmanager_2.
2019-10-09">flink@10.160.140.216:26073/user/jobmanager_2.
2019-10-09 13:54:21.805 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock.
2019-10-09 13:54:21.806 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/jobmanager_2 no longer participates in the leader election.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher was revoked leadership.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-80] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Stopping all currently running jobs of dispatcher akka.tcp://<a href="http://flink@10.160.140.216:26073/user/dispatcher.
2019-10-09">flink@10.160.140.216:26073/user/dispatcher.
2019-10-09 13:54:21.806 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/dispatcher no longer participates in the leader election.
2019-10-09 13:54:21.808 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) switched from state RUNNING to SUSPENDING.
org.apache.flink.util.FlinkException: JobManager is no longer the leader.
	at org.apache.flink.runtime.jobmaster.JobManagerRunner.revokeLeadership(JobManagerRunner.java:371)
	at org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService.notLeader(ZooKeeperLeaderElectionService.java:247)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$8.apply(LeaderLatch.java:640)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$8.apply(LeaderLatch.java:636)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93)
	at org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.setLeadership(LeaderLatch.java:635)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.handleStateChange(LeaderLatch.java:623)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.access$000(LeaderLatch.java:64)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$1.stateChanged(LeaderLatch.java:82)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:259)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:255)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93)
	at org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager.processEvents(ConnectionStateManager.java:253)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager.access$000(ConnectionStateManager.java:43)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$1.call(ConnectionStateManager.java:111)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-88] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Suspending the SlotManager.
2019-10-09 13:54:21.808 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.809 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/resourcemanager no longer participates in the leader election.
2019-10-09 13:54:21.819 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (1/8) (4a825def14a88297d19c9e5738dbf668) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (2/8) (6517a467aaf7be56bfb3821b29852e20) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (3/8) (7a7ee5b1f3613b97d858564dbfaeb066) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (4/8) (2d878a7c86e9d6e0bb0fe737a42037e5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.821 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (5/8) (82b545cdf0918a7d14b8c595884a5caf) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.821 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (6/8) (fb178b7aed87e2fc7c8be11838575cb6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (7/8) (7d2956af9eca83b125a5f2e3d65c7454) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (8/8) (8673406c095fbb33ff63b8116c2dc6c4) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (1/60) (fde9e604f69f27dd1e1afc2d6f96d6a6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (2/60) (44429fa0423a6c4e2cfa73b7caf98e7f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (3/60) (ccb886b9e7f7f3e9391a66a77a95d936) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (4/60) (9dfbcca1af51c7565d082ae1d4fbca99) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (5/60) (984560fc276a70ffb3566ec5236a57d6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (6/60) (42c8e83ec209f1947dfc53a3451ca002) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (7/60) (2fff02384094c4889f8d583db6c7c495) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (8/60) (bd81e085522ef4915a9413517dcb51ec) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (9/60) (11762d1b367d402bca99be0624d637c6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (10/60) (bcba40c174b64e16b1a48e802e0251c8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (11/60) (5655a9923af5f1d40436e2531ca87ff9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (12/60) (b9e3017ec604802c151939077f6f4f46) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (13/60) (4fee14cb460433aa6178fe98315631ff) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (14/60) (ec80588d10b6a11cc52625f95ebf4858) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (15/60) (5d55b130964c3c164df9938bb9b9b8e9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (16/60) (b67388f779efeb6ffcae57aaaf0e992f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (17/60) (c91c36736380adc280b7b9235bd26609) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (18/60) (f658b30a9c2b4dee63de0fb4797d0c3b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (19/60) (bb92a17d4bd6279729ac6e9274ca80ee) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (20/60) (ecf8a4563990fba29df91ee769e5ea2e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (21/60) (8ea7047b457ad4bded12ea4dcea1576b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (22/60) (a2cea7a46ecffbd5275d355a29bc09f9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (23/60) (105456dc1d11fc9d8164ea0d54794d90) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (24/60) (faa255028f7bba0ef8b832616a09e0cc) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (25/60) (80a6454bca3392e1050b02c2b689edae) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (26/60) (de3561bf14ed9617a4727491ad959219) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (27/60) (585101b822efc77260c0f0a52eb5f856) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (28/60) (3205706648aa73f9e34d29c22428ff90) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (29/60) (24dd134cb75696784bade727d6c57459) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (30/60) (7d21e0c3934c4bfdc5349f0627fdd923) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (31/60) (07672677de03585cc630e1014143bec8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (32/60) (72159fe698efbaacfb4690b22e4dad03) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (33/60) (a2f3dce63aac51c82060f02646019af5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (34/60) (c5e4f81b8c2590d0f00c7a5c3c05f7a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (35/60) (87579d8547b643a5e1418dca333b426f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (36/60) (021f60ef60686370a2e9e6d92fa033ad) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (37/60) (65b2647fabcbfb1b23aaea63d4627509) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (38/60) (3f054f2dc5161bd67bbec03d711dea18) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (39/60) (9b7f93456479e1943de63da8ff8c88d3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (40/60) (f2c204e511ed878b8bbd9fded928c215) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (41/60) (e139c0820ecd8cb8c8fe168b9a847a1e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (42/60) (42b961618e0fe859f0761e50e0569113) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (43/60) (29928821d6f94e715cc2ab444880fb7b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (44/60) (83e9b131ba8684474e13236b67916ac3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (45/60) (fe3e5b53d7fbc4e795822944317e85bb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (46/60) (d7617e20894c359f95cd363faaa3b117) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (47/60) (e7545083f44b6a8338750542943d67f0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (48/60) (0fc95f3ca9dafaeb8d90cd262de82440) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (49/60) (aa41f35f708625f08bde4b3513e3d769) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (50/60) (4d01c8d2543d86f2cdd89db1b81df2af) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (51/60) (6d5314d1d924da1437dfee4ff243ffb9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (52/60) (fad4dc6e3ba4d0be2abbecb7a960c914) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (53/60) (0ccd9b09fc32f61b45031906a7c612b0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (54/60) (1fbc38dc9181a399bd30de4e4044e261) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (55/60) (2ea15174000a056b0d747d1089f6585f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (56/60) (28baca2d42222f0a0182c5c89e1717c6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (57/60) (e1abb8bce0113fb035f4a36271b73f17) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (58/60) (9f4a99aad3d396dbbffc75bc9b40f0b6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (59/60) (d58fd6d1d88118d0037c3c2e92176b69) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (60/60) (765e9de291878316b4c69000bcd0b2df) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (1/30) (782206df8a2ad0f7a2946eedf0cbc827) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (2/30) (db788aa1e9fdb0022bf9d5a7dc257235) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (3/30) (8567c954df44048c35632a52314d9660) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (4/30) (b8d2a0f72c717e2a06e5770751bf20e7) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (5/30) (d72b93dbfa363c38bd6b7451a39c7d67) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (6/30) (559d7f2c0bcb8255df8963b3b4329f66) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (7/30) (8124a163b7727ce53601ba21c7a49acc) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (8/30) (c151f0d09157158e3f449a847c41cc65) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (9/30) (86b2c2178b48d7bb74742036b6489260) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (10/30) (f716beb167fbe51c1db7798c1f1794d3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (11/30) (b72e2b66c4c3fa24a2e6d247210f9e34) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (12/30) (e63a7ed9a8fb828a1cda0e206fa803e1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (13/30) (7dcb47fd7e88bdc86ae45344d9779e99) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (14/30) (ccc164979332f39f374664d758345301) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (15/30) (b21287c5c6b9b89124a1cd6810eb2a21) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (16/30) (91071077d5e04266d5b6d9833cf2f9e4) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (17/30) (b5ec2ff0c6decca89ff76223c25b9d9b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (18/30) (0774604957ec0d2680b8673b98dbbe7b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (19/30) (6e402ae9f4d0b8755c0c7a2ef67f2c1a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (20/30) (51493f15a4b7166aa390f854a597c4f6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (21/30) (e329d85d95a8a6a142b8157bd3b2eeee) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (22/30) (c17a752c676bd4fa1559b6641f959fc7) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (23/30) (94bfdd3b1bb3be163c951b41aa80f5a8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (24/30) (f4a378189dd403de5d4d5a3e77415f9e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (25/30) (d16ff3ac286ab1fd32caa9f46fca5619) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (26/30) (090c3aaa6bb23261e74a7bf70060a36d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (27/30) (9aeebb1f3b6d51be5388674955953c52) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (28/30) (23c2fceab79f540a9d89e2e4f092a865) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (29/30) (e6bb884157821902ae36d036737efaa5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (30/30) (afca1c29fa6c50a49c88d59d3df459f6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (1/30) (9adb75badd9cb8afbb4251a33ef27171) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (2/30) (473835002699bfb37c5b88edafd219a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (3/30) (e7e030cb12de97e633cf473d7e627206) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (4/30) (0459e2f2ea079437f62769f2aee861d2) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (5/30) (ed428b67cd250fd6407a0b5243aab514) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (6/30) (4a54f08765d6c9faac31ace85a406819) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (7/30) (24ab50af8fdab9de5f0915479cedc50f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (8/30) (5f365913e8e69f0bb8fad4fcac0337ea) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (9/30) (d4ac35d28253d1dd096fa6bab4fafc8a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (10/30) (1ad00f44a299183f81f61ed829410b1d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (11/30) (98238896820682b892815dfa2f62e6c2) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (12/30) (6019139a62f395abb0d3bb0fa51b326e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (13/30) (73f72dd701d042dacd95988521873013) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (14/30) (3109dae5918d772b64e7d8b28b128dfe) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (15/30) (10c0effec4da40afe144a59556882680) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (16/30) (722f04d080fed0df95787fc11a7c95d0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (17/30) (07461d9203eedf28c0a8c893a631ca2f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (18/30) (206c625688c12fa82cfc3bae7b60f24d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (19/30) (6fd477a3d452423a09cd0ba0494a6deb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (20/30) (d0aa4c46dbb46db09c9395e39b41a4c1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (21/30) (0dde7060b76feb5e7333e20d3fe4e7a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (22/30) (ce6749c3e5b996df8647603e4e8fbfdb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (23/30) (8270b98412ef6613859f9ae0109850e3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (24/30) (136806c81715dd66a3d5cf8f49ceb031) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (25/30) (e95b22dff10204581a86a35461bf5768) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (26/30) (9fff9a9c688f9e383f18661923a814ba) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (27/30) (2626c8dad0fbab32fadd995930211d8a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (28/30) (70aa1e618f77758c684cae15b7722752) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (29/30) (ef3392ed8088af2c05212e0fb0a5aac6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.853 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (30/30) (6af3938acd0f635d95c9c432edeab539) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.855 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Close ResourceManager connection 490c7cf7fc01baf7e0ece283d48690a4: JobManager is no longer the leader..
2019-10-09 13:54:21.855 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Suspending SlotPool.
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Stopping the JobMaster for job hdp-shoujiserver_CommercialDotingAccess(2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Stopping SlotPool.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Stopping ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock'}.
2019-10-09 13:54:22.602 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.603 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.29/10.160.101.29:2181
2019-10-09">10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:22.603 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.603 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:22.921 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.922 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.28/10.160.101.28:2181
2019-10-09">10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:22.922 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.922 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.008 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.008 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.26/10.160.101.26:2181
2019-10-09">10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:24.008 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.009 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.778 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.778 [main-SendThread(10.160.101.30:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.30/10.160.101.30:2181
2019-10-09">10.160.101.30/10.160.101.30:2181
2019-10-09 13:54:24.778 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.779 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:26.748 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:26.748 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.27/10.160.101.27:2181
2019-10-09">10.160.101.27/10.160.101.27:2181
2019-10-09 13:54:26.749 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:26.749 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.27/10.160.101.27:2181, initiating session
2019-10-09 13:54:26.751 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:27.673 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:27.674 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.29/10.160.101.29:2181
2019-10-09">10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:27.674 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:27.674 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.29/10.160.101.29:2181, initiating session
2019-10-09 13:54:27.675 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:27.865 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:27.865 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.28/10.160.101.28:2181
2019-10-09">10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:27.865 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:27.866 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.28/10.160.101.28:2181, initiating session
2019-10-09 13:54:27.867 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:28.078 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:28.078 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server <a href="http://10.160.101.26/10.160.101.26:2181
2019-10-09">10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:28.079 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:28.079 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.26/10.160.101.26:2181, initiating session
2019-10-09 13:54:28.080 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session establishment complete on server 10.160.101.26/10.160.101.26:2181, sessionid = 0x4597758a3589ca6, negotiated timeout = 40000
2019-10-09 13:54:28.081 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: RECONNECTED
2019-10-09 13:54:28.081 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.082 [Curator-PathChildrenCache-0] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - ZooKeeper connection RECONNECTED. Changes to the submitted job graphs are monitored again.
2019-10-09 13:54:28.082 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:28.082 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.083 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:28.083 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.087 [flink-akka.actor.default-dispatcher-88] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 2657b25ff073a91c888ff78a28bd2331 from ZooKeeper.
2019-10-09 13:54:28.088 [main-EventThread] INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint  - http://10.160.140.216:52303 was granted leadership with leaderSessionID=d80d351c-56ca-490f-aa78-681847d58cfe
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-83] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.160.140.216:26073/user/resourcemanager was granted leadership with fencing token 8b328120280016029994e898e40749f8
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-83] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Starting the SlotManager.
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-48] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher was granted leadership with fencing token 65d868d3-0665-4555-ae75-86975bb3c7a9
2019-10-09 13:54:28.093 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Recovering all persisted jobs.
2019-10-09 13:54:28.130 [flink-akka.actor.default-dispatcher-72] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Recovered SubmittedJobGraph(2657b25ff073a91c888ff78a28bd2331, null).
2019-10-09 13:54:28.212 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000003 (akka.tcp://flink@10.160.167.236:37058/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:28.515 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000005 (akka.tcp://flink@10.162.92.119:34318/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.072 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000008 (akka.tcp://flink@10.160.132.173:48920/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.473 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  - Starting RPC endpoint for org.apache.flink.runtime.jobmaster.JobMaster at akka://flink/user/jobmanager_3 .
2019-10-09 13:54:29.474 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Initializing job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.487 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Using restart strategy FailureRateRestartStrategy(failuresInterval=3600000 msdelayInterval=30000 msmaxFailuresPerInterval=3) for hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.489 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  - Starting RPC endpoint for org.apache.flink.runtime.jobmaster.slotpool.SlotPool at akka://flink/user/25b2e77f-fe0c-4e2d-af38-8900a36077d7 .
2019-10-09 13:54:29.489 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job recovers via failover strategy: full graph restart
2019-10-09 13:54:29.490 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Running initialization on master for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.490 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Successfully ran initialization on master in 0 ms.
2019-10-09 13:54:29.497 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Initialized in '/checkpoints/2657b25ff073a91c888ff78a28bd2331'.
2019-10-09 13:54:29.498 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Loading state backend via factory org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory
2019-10-09 13:54:29.498 [flink-akka.actor.default-dispatcher-88] WARN  org.apache.flink.configuration.Configuration  - Config uses deprecated configuration key 'state.backend.rocksdb.checkpointdir' instead of proper key 'state.backend.rocksdb.localdir'
2019-10-09 13:54:29.502 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Recovering checkpoints from ZooKeeper.
2019-10-09 13:54:29.513 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Found 5 checkpoints in ZooKeeper.
2019-10-09 13:54:29.514 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to fetch 5 checkpoints from storage.
2019-10-09 13:54:29.514 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33036.
2019-10-09 13:54:29.516 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000010 (akka.tcp://flink@10.160.124.173:30267/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.523 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33037.
2019-10-09 13:54:29.532 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33038.
2019-10-09 13:54:29.540 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33039.
2019-10-09 13:54:29.549 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33040.
2019-10-09 13:54:30.712 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - Restoring job 2657b25ff073a91c888ff78a28bd2331 from latest valid checkpoint: Checkpoint 33040 @ 1570600391503 for 2657b25ff073a91c888ff78a28bd2331.
2019-10-09 13:54:30.715 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - No master state to restore
2019-10-09 13:54:30.715 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock'}.
2019-10-09 13:54:30.721 [main-EventThread] INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner  - JobManager runner for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) was granted leadership with session id 829a1a30-be5a-4e4a-aa0c-21cc71b45599 at akka.tcp://<a href="http://flink@10.160.140.216:26073/user/jobmanager_3.
2019-10-09">flink@10.160.140.216:26073/user/jobmanager_3.
2019-10-09 13:54:30.724 [flink-akka.actor.default-dispatcher-21] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2019-10-09 13:54:30.724 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Starting execution of job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331)
2019-10-09 13:54:30.725 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) switched from state CREATED to RUNNING.
2019-10-09 13:54:30.725 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (1/8) (d072a1bc3f3dbf7896ef619951d91c88) switched from CREATED to SCHEDULED.
After the zk reconnected after a suspend, the job was recovered, and you can see, there was a StandaloneDispatcher created.


Don't know why the 'FencingTokenException: Fencing token not set' exception happened, and don't know why different dispatchers were created.

Thanks

Joshua


On Wed, Oct 2, 2019 at 9:10 PM Till Rohrmann <[hidden email]> wrote:
Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

Cheers,
Till

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:
Hi Bruce,

I haven't seen such an exception yet, but maybe Till (in CC) can help.

Best,
Fabian

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 

Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Till Rohrmann
Hi Joshua,

the FencingTokenException should not be a problem because they originate from rest handler which send requests to the Dispatcher before it has accepted the leadership. The question I would ask is what is the MiniDispatcher doing after it received leadership. Normally it should try to recover all persisted jobs. However, this is not the case. Therefore I assume that you might run into FLINK-11843 [1]. In order to substantiate this suspicion, I would need to see what happened just before the cluster lost the connection to ZooKeeper.

Which version of Flink are you running btw?


Cheers,
Till

On Thu, Oct 10, 2019 at 11:31 AM Joshua Fan <[hidden email]> wrote:
Hi Till

Recently, this issue happened frequently, but some jobs recovered from the exception, some not.

for the job who can not recover from the exception, the log pasted below:

2019-10-09 13:54:21.746 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: SUSPENDED
2019-10-09 13:54:21.746 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.162.89.153:16736/user/dispatcher no longer participates in the leader election.
2019-10-09 13:54:21.746 [flink-akka.actor.default-dispatcher-46] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.162.89.153:16736/user/resourcemanager was revoked leadership. Clearing fencing token.
2019-10-09 13:54:21.746 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender http://10.162.89.153:12503 no longer participates in the leader election.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] INFO  o.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint  - http://10.162.89.153:12503 lost leadership
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.162.89.153:16736/user/resourcemanager no longer participates in the leader election.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.747 [flink-akka.actor.default-dispatcher-43] INFO  org.apache.flink.runtime.dispatcher.MiniDispatcher  - Dispatcher akka.tcp://flink@10.162.89.153:16736/user/dispatcher was revoked leadership.
2019-10-09 13:54:21.747 [flink-akka.actor.default-dispatcher-46] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Suspending the SlotManager.
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:22.072 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:22.973 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.973 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.27/10.160.101.27:2181
2019-10-09 13:54:22.973 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.974 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:23.405 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:23.405 [main-SendThread(10.160.101.30:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.30/10.160.101.30:2181
2019-10-09 13:54:23.405 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:23.406 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.379 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.379 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:24.379 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.380 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:26.044 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:26.045 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:26.045 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:29.050 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.29/10.160.101.29:2181, initiating session
2019-10-09 13:54:29.095 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session establishment complete on server 10.160.101.29/10.160.101.29:2181, sessionid = 0x2597758ae5091ff, negotiated timeout = 40000
2019-10-09 13:54:29.095 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: RECONNECTED
2019-10-09 13:54:29.096 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:29.105 [flink-akka.actor.default-dispatcher-46] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.162.89.153:16736/user/resourcemanager was granted leadership with fencing token a22e4c9c80fd0c7249102f69c0b64e01
2019-10-09 13:54:29.105 [flink-akka.actor.default-dispatcher-46] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Starting the SlotManager.
2019-10-09 13:54:29.107 [main-EventThread] INFO  o.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint  - http://10.162.89.153:12503 was granted leadership with leaderSessionID=53b02e00-95c3-4d34-a272-b84795b991a7
2019-10-09 13:54:29.116 [flink-akka.actor.default-dispatcher-26] INFO  org.apache.flink.runtime.dispatcher.MiniDispatcher  - Dispatcher akka.tcp://flink@10.162.89.153:16736/user/dispatcher was granted leadership with fencing token d645a3fe-8fee-453b-abd5-bd4cd82ebba1
2019-10-09 13:54:34.809 [flink-rest-server-netty-worker-thread-73] ERROR org.apache.flink.runtime.rest.handler.job.JobDetailsHandler  - Could not retrieve the redirect address.
java.util.concurrent.CompletionException: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:772)
	at akka.dispatch.OnComplete.internal(Future.scala:258)
	at akka.dispatch.OnComplete.internal(Future.scala:256)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
	at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
	at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
	at akka.actor.ActorRef.tell(ActorRef.scala:130)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendErrorIfSender(AkkaRpcActor.java:371)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:57)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)
	at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)
	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)
Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)
	... 14 common frames omitted
2019-10-09 13:54:56.007 [flink-rest-server-netty-worker-thread-74] ERROR org.apache.flink.runtime.rest.handler.job.JobDetailsHandler  - Could not retrieve the redirect address.
java.util.concurrent.CompletionException: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:772)
	at akka.dispatch.OnComplete.internal(Future.scala:258)
	at akka.dispatch.OnComplete.internal(Future.scala:256)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
	at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
	at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
	at akka.actor.ActorRef.tell(ActorRef.scala:130)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendErrorIfSender(AkkaRpcActor.java:371)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:57)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)
	at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)
	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)
Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)
	... 14 common frames omitted

In the above log, the ha zk got reconnected after a suspend, but the job can not recover, and there existed a MiniDispatcherRestEndpoint.


for the job who can recover from the same condition, the log pasted as below:

2019-10-09 13:54:21.803 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: SUSPENDED
2019-10-09 13:54:21.804 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender http://10.160.140.216:52303 no longer participates in the leader election.
2019-10-09 13:54:21.804 [Curator-ConnectionStateManager-0] INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint  - http://10.160.140.216:52303 lost leadership
2019-10-09 13:54:21.804 [Curator-PathChildrenCache-0] WARN  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - ZooKeeper connection SUSPENDING. Changes to the submitted job graphs are not monitored (temporarily).
2019-10-09 13:54:21.805 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.805 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.160.140.216:26073/user/resourcemanager was revoked leadership. Clearing fencing token.
2019-10-09 13:54:21.805 [Curator-ConnectionStateManager-0] INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner  - JobManager for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) was revoked leadership at akka.tcp://flink@10.160.140.216:26073/user/jobmanager_2.
2019-10-09 13:54:21.805 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock.
2019-10-09 13:54:21.806 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/jobmanager_2 no longer participates in the leader election.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher was revoked leadership.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-80] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Stopping all currently running jobs of dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher.
2019-10-09 13:54:21.806 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/dispatcher no longer participates in the leader election.
2019-10-09 13:54:21.808 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) switched from state RUNNING to SUSPENDING.
org.apache.flink.util.FlinkException: JobManager is no longer the leader.
	at org.apache.flink.runtime.jobmaster.JobManagerRunner.revokeLeadership(JobManagerRunner.java:371)
	at org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService.notLeader(ZooKeeperLeaderElectionService.java:247)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$8.apply(LeaderLatch.java:640)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$8.apply(LeaderLatch.java:636)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93)
	at org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.setLeadership(LeaderLatch.java:635)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.handleStateChange(LeaderLatch.java:623)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.access$000(LeaderLatch.java:64)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$1.stateChanged(LeaderLatch.java:82)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:259)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:255)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93)
	at org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager.processEvents(ConnectionStateManager.java:253)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager.access$000(ConnectionStateManager.java:43)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$1.call(ConnectionStateManager.java:111)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-88] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Suspending the SlotManager.
2019-10-09 13:54:21.808 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.809 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/resourcemanager no longer participates in the leader election.
2019-10-09 13:54:21.819 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (1/8) (4a825def14a88297d19c9e5738dbf668) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (2/8) (6517a467aaf7be56bfb3821b29852e20) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (3/8) (7a7ee5b1f3613b97d858564dbfaeb066) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (4/8) (2d878a7c86e9d6e0bb0fe737a42037e5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.821 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (5/8) (82b545cdf0918a7d14b8c595884a5caf) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.821 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (6/8) (fb178b7aed87e2fc7c8be11838575cb6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (7/8) (7d2956af9eca83b125a5f2e3d65c7454) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (8/8) (8673406c095fbb33ff63b8116c2dc6c4) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (1/60) (fde9e604f69f27dd1e1afc2d6f96d6a6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (2/60) (44429fa0423a6c4e2cfa73b7caf98e7f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (3/60) (ccb886b9e7f7f3e9391a66a77a95d936) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (4/60) (9dfbcca1af51c7565d082ae1d4fbca99) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (5/60) (984560fc276a70ffb3566ec5236a57d6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (6/60) (42c8e83ec209f1947dfc53a3451ca002) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (7/60) (2fff02384094c4889f8d583db6c7c495) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (8/60) (bd81e085522ef4915a9413517dcb51ec) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (9/60) (11762d1b367d402bca99be0624d637c6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (10/60) (bcba40c174b64e16b1a48e802e0251c8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (11/60) (5655a9923af5f1d40436e2531ca87ff9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (12/60) (b9e3017ec604802c151939077f6f4f46) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (13/60) (4fee14cb460433aa6178fe98315631ff) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (14/60) (ec80588d10b6a11cc52625f95ebf4858) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (15/60) (5d55b130964c3c164df9938bb9b9b8e9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (16/60) (b67388f779efeb6ffcae57aaaf0e992f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (17/60) (c91c36736380adc280b7b9235bd26609) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (18/60) (f658b30a9c2b4dee63de0fb4797d0c3b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (19/60) (bb92a17d4bd6279729ac6e9274ca80ee) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (20/60) (ecf8a4563990fba29df91ee769e5ea2e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (21/60) (8ea7047b457ad4bded12ea4dcea1576b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (22/60) (a2cea7a46ecffbd5275d355a29bc09f9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (23/60) (105456dc1d11fc9d8164ea0d54794d90) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (24/60) (faa255028f7bba0ef8b832616a09e0cc) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (25/60) (80a6454bca3392e1050b02c2b689edae) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (26/60) (de3561bf14ed9617a4727491ad959219) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (27/60) (585101b822efc77260c0f0a52eb5f856) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (28/60) (3205706648aa73f9e34d29c22428ff90) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (29/60) (24dd134cb75696784bade727d6c57459) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (30/60) (7d21e0c3934c4bfdc5349f0627fdd923) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (31/60) (07672677de03585cc630e1014143bec8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (32/60) (72159fe698efbaacfb4690b22e4dad03) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (33/60) (a2f3dce63aac51c82060f02646019af5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (34/60) (c5e4f81b8c2590d0f00c7a5c3c05f7a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (35/60) (87579d8547b643a5e1418dca333b426f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (36/60) (021f60ef60686370a2e9e6d92fa033ad) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (37/60) (65b2647fabcbfb1b23aaea63d4627509) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (38/60) (3f054f2dc5161bd67bbec03d711dea18) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (39/60) (9b7f93456479e1943de63da8ff8c88d3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (40/60) (f2c204e511ed878b8bbd9fded928c215) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (41/60) (e139c0820ecd8cb8c8fe168b9a847a1e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (42/60) (42b961618e0fe859f0761e50e0569113) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (43/60) (29928821d6f94e715cc2ab444880fb7b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (44/60) (83e9b131ba8684474e13236b67916ac3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (45/60) (fe3e5b53d7fbc4e795822944317e85bb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (46/60) (d7617e20894c359f95cd363faaa3b117) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (47/60) (e7545083f44b6a8338750542943d67f0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (48/60) (0fc95f3ca9dafaeb8d90cd262de82440) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (49/60) (aa41f35f708625f08bde4b3513e3d769) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (50/60) (4d01c8d2543d86f2cdd89db1b81df2af) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (51/60) (6d5314d1d924da1437dfee4ff243ffb9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (52/60) (fad4dc6e3ba4d0be2abbecb7a960c914) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (53/60) (0ccd9b09fc32f61b45031906a7c612b0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (54/60) (1fbc38dc9181a399bd30de4e4044e261) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (55/60) (2ea15174000a056b0d747d1089f6585f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (56/60) (28baca2d42222f0a0182c5c89e1717c6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (57/60) (e1abb8bce0113fb035f4a36271b73f17) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (58/60) (9f4a99aad3d396dbbffc75bc9b40f0b6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (59/60) (d58fd6d1d88118d0037c3c2e92176b69) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (60/60) (765e9de291878316b4c69000bcd0b2df) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (1/30) (782206df8a2ad0f7a2946eedf0cbc827) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (2/30) (db788aa1e9fdb0022bf9d5a7dc257235) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (3/30) (8567c954df44048c35632a52314d9660) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (4/30) (b8d2a0f72c717e2a06e5770751bf20e7) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (5/30) (d72b93dbfa363c38bd6b7451a39c7d67) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (6/30) (559d7f2c0bcb8255df8963b3b4329f66) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (7/30) (8124a163b7727ce53601ba21c7a49acc) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (8/30) (c151f0d09157158e3f449a847c41cc65) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (9/30) (86b2c2178b48d7bb74742036b6489260) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (10/30) (f716beb167fbe51c1db7798c1f1794d3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (11/30) (b72e2b66c4c3fa24a2e6d247210f9e34) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (12/30) (e63a7ed9a8fb828a1cda0e206fa803e1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (13/30) (7dcb47fd7e88bdc86ae45344d9779e99) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (14/30) (ccc164979332f39f374664d758345301) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (15/30) (b21287c5c6b9b89124a1cd6810eb2a21) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (16/30) (91071077d5e04266d5b6d9833cf2f9e4) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (17/30) (b5ec2ff0c6decca89ff76223c25b9d9b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (18/30) (0774604957ec0d2680b8673b98dbbe7b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (19/30) (6e402ae9f4d0b8755c0c7a2ef67f2c1a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (20/30) (51493f15a4b7166aa390f854a597c4f6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (21/30) (e329d85d95a8a6a142b8157bd3b2eeee) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (22/30) (c17a752c676bd4fa1559b6641f959fc7) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (23/30) (94bfdd3b1bb3be163c951b41aa80f5a8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (24/30) (f4a378189dd403de5d4d5a3e77415f9e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (25/30) (d16ff3ac286ab1fd32caa9f46fca5619) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (26/30) (090c3aaa6bb23261e74a7bf70060a36d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (27/30) (9aeebb1f3b6d51be5388674955953c52) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (28/30) (23c2fceab79f540a9d89e2e4f092a865) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (29/30) (e6bb884157821902ae36d036737efaa5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (30/30) (afca1c29fa6c50a49c88d59d3df459f6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (1/30) (9adb75badd9cb8afbb4251a33ef27171) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (2/30) (473835002699bfb37c5b88edafd219a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (3/30) (e7e030cb12de97e633cf473d7e627206) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (4/30) (0459e2f2ea079437f62769f2aee861d2) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (5/30) (ed428b67cd250fd6407a0b5243aab514) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (6/30) (4a54f08765d6c9faac31ace85a406819) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (7/30) (24ab50af8fdab9de5f0915479cedc50f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (8/30) (5f365913e8e69f0bb8fad4fcac0337ea) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (9/30) (d4ac35d28253d1dd096fa6bab4fafc8a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (10/30) (1ad00f44a299183f81f61ed829410b1d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (11/30) (98238896820682b892815dfa2f62e6c2) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (12/30) (6019139a62f395abb0d3bb0fa51b326e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (13/30) (73f72dd701d042dacd95988521873013) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (14/30) (3109dae5918d772b64e7d8b28b128dfe) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (15/30) (10c0effec4da40afe144a59556882680) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (16/30) (722f04d080fed0df95787fc11a7c95d0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (17/30) (07461d9203eedf28c0a8c893a631ca2f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (18/30) (206c625688c12fa82cfc3bae7b60f24d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (19/30) (6fd477a3d452423a09cd0ba0494a6deb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (20/30) (d0aa4c46dbb46db09c9395e39b41a4c1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (21/30) (0dde7060b76feb5e7333e20d3fe4e7a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (22/30) (ce6749c3e5b996df8647603e4e8fbfdb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (23/30) (8270b98412ef6613859f9ae0109850e3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (24/30) (136806c81715dd66a3d5cf8f49ceb031) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (25/30) (e95b22dff10204581a86a35461bf5768) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (26/30) (9fff9a9c688f9e383f18661923a814ba) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (27/30) (2626c8dad0fbab32fadd995930211d8a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (28/30) (70aa1e618f77758c684cae15b7722752) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (29/30) (ef3392ed8088af2c05212e0fb0a5aac6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.853 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (30/30) (6af3938acd0f635d95c9c432edeab539) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.855 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Close ResourceManager connection 490c7cf7fc01baf7e0ece283d48690a4: JobManager is no longer the leader..
2019-10-09 13:54:21.855 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Suspending SlotPool.
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Stopping the JobMaster for job hdp-shoujiserver_CommercialDotingAccess(2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Stopping SlotPool.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Stopping ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock'}.
2019-10-09 13:54:22.602 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.603 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:22.603 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.603 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:22.921 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.922 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:22.922 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.922 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.008 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.008 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:24.008 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.009 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.778 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.778 [main-SendThread(10.160.101.30:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.30/10.160.101.30:2181
2019-10-09 13:54:24.778 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.779 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:26.748 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:26.748 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.27/10.160.101.27:2181
2019-10-09 13:54:26.749 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:26.749 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.27/10.160.101.27:2181, initiating session
2019-10-09 13:54:26.751 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:27.673 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:27.674 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:27.674 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:27.674 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.29/10.160.101.29:2181, initiating session
2019-10-09 13:54:27.675 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:27.865 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:27.865 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:27.865 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:27.866 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.28/10.160.101.28:2181, initiating session
2019-10-09 13:54:27.867 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:28.078 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:28.078 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:28.079 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:28.079 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.26/10.160.101.26:2181, initiating session
2019-10-09 13:54:28.080 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session establishment complete on server 10.160.101.26/10.160.101.26:2181, sessionid = 0x4597758a3589ca6, negotiated timeout = 40000
2019-10-09 13:54:28.081 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: RECONNECTED
2019-10-09 13:54:28.081 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.082 [Curator-PathChildrenCache-0] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - ZooKeeper connection RECONNECTED. Changes to the submitted job graphs are monitored again.
2019-10-09 13:54:28.082 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:28.082 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.083 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:28.083 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.087 [flink-akka.actor.default-dispatcher-88] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 2657b25ff073a91c888ff78a28bd2331 from ZooKeeper.
2019-10-09 13:54:28.088 [main-EventThread] INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint  - http://10.160.140.216:52303 was granted leadership with leaderSessionID=d80d351c-56ca-490f-aa78-681847d58cfe
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-83] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.160.140.216:26073/user/resourcemanager was granted leadership with fencing token 8b328120280016029994e898e40749f8
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-83] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Starting the SlotManager.
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-48] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher was granted leadership with fencing token 65d868d3-0665-4555-ae75-86975bb3c7a9
2019-10-09 13:54:28.093 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Recovering all persisted jobs.
2019-10-09 13:54:28.130 [flink-akka.actor.default-dispatcher-72] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Recovered SubmittedJobGraph(2657b25ff073a91c888ff78a28bd2331, null).
2019-10-09 13:54:28.212 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000003 (akka.tcp://flink@10.160.167.236:37058/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:28.515 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000005 (akka.tcp://flink@10.162.92.119:34318/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.072 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000008 (akka.tcp://flink@10.160.132.173:48920/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.473 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  - Starting RPC endpoint for org.apache.flink.runtime.jobmaster.JobMaster at akka://flink/user/jobmanager_3 .
2019-10-09 13:54:29.474 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Initializing job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.487 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Using restart strategy FailureRateRestartStrategy(failuresInterval=3600000 msdelayInterval=30000 msmaxFailuresPerInterval=3) for hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.489 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  - Starting RPC endpoint for org.apache.flink.runtime.jobmaster.slotpool.SlotPool at akka://flink/user/25b2e77f-fe0c-4e2d-af38-8900a36077d7 .
2019-10-09 13:54:29.489 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job recovers via failover strategy: full graph restart
2019-10-09 13:54:29.490 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Running initialization on master for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.490 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Successfully ran initialization on master in 0 ms.
2019-10-09 13:54:29.497 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Initialized in '/checkpoints/2657b25ff073a91c888ff78a28bd2331'.
2019-10-09 13:54:29.498 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Loading state backend via factory org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory
2019-10-09 13:54:29.498 [flink-akka.actor.default-dispatcher-88] WARN  org.apache.flink.configuration.Configuration  - Config uses deprecated configuration key 'state.backend.rocksdb.checkpointdir' instead of proper key 'state.backend.rocksdb.localdir'
2019-10-09 13:54:29.502 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Recovering checkpoints from ZooKeeper.
2019-10-09 13:54:29.513 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Found 5 checkpoints in ZooKeeper.
2019-10-09 13:54:29.514 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to fetch 5 checkpoints from storage.
2019-10-09 13:54:29.514 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33036.
2019-10-09 13:54:29.516 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000010 (akka.tcp://flink@10.160.124.173:30267/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.523 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33037.
2019-10-09 13:54:29.532 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33038.
2019-10-09 13:54:29.540 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33039.
2019-10-09 13:54:29.549 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33040.
2019-10-09 13:54:30.712 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - Restoring job 2657b25ff073a91c888ff78a28bd2331 from latest valid checkpoint: Checkpoint 33040 @ 1570600391503 for 2657b25ff073a91c888ff78a28bd2331.
2019-10-09 13:54:30.715 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - No master state to restore
2019-10-09 13:54:30.715 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock'}.
2019-10-09 13:54:30.721 [main-EventThread] INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner  - JobManager runner for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) was granted leadership with session id 829a1a30-be5a-4e4a-aa0c-21cc71b45599 at akka.tcp://flink@10.160.140.216:26073/user/jobmanager_3.
2019-10-09 13:54:30.724 [flink-akka.actor.default-dispatcher-21] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2019-10-09 13:54:30.724 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Starting execution of job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331)
2019-10-09 13:54:30.725 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) switched from state CREATED to RUNNING.
2019-10-09 13:54:30.725 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (1/8) (d072a1bc3f3dbf7896ef619951d91c88) switched from CREATED to SCHEDULED.
After the zk reconnected after a suspend, the job was recovered, and you can see, there was a StandaloneDispatcher created.


Don't know why the 'FencingTokenException: Fencing token not set' exception happened, and don't know why different dispatchers were created.

Thanks

Joshua


On Wed, Oct 2, 2019 at 9:10 PM Till Rohrmann <[hidden email]> wrote:
Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

Cheers,
Till

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:
Hi Bruce,

I haven't seen such an exception yet, but maybe Till (in CC) can help.

Best,
Fabian

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 

Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

tison
Hi Till,

I'd like to re-throw FLINK-10052 that tolerate ConnectionLossException of zookeeper so that we make it quite rare user encounter this kind of issues.

Best,
tison.


Till Rohrmann <[hidden email]> 于2019年10月10日周四 下午10:40写道:
Hi Joshua,

the FencingTokenException should not be a problem because they originate from rest handler which send requests to the Dispatcher before it has accepted the leadership. The question I would ask is what is the MiniDispatcher doing after it received leadership. Normally it should try to recover all persisted jobs. However, this is not the case. Therefore I assume that you might run into FLINK-11843 [1]. In order to substantiate this suspicion, I would need to see what happened just before the cluster lost the connection to ZooKeeper.

Which version of Flink are you running btw?


Cheers,
Till

On Thu, Oct 10, 2019 at 11:31 AM Joshua Fan <[hidden email]> wrote:
Hi Till

Recently, this issue happened frequently, but some jobs recovered from the exception, some not.

for the job who can not recover from the exception, the log pasted below:

2019-10-09 13:54:21.746 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: SUSPENDED
2019-10-09 13:54:21.746 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.162.89.153:16736/user/dispatcher no longer participates in the leader election.
2019-10-09 13:54:21.746 [flink-akka.actor.default-dispatcher-46] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.162.89.153:16736/user/resourcemanager was revoked leadership. Clearing fencing token.
2019-10-09 13:54:21.746 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender http://10.162.89.153:12503 no longer participates in the leader election.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] INFO  o.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint  - http://10.162.89.153:12503 lost leadership
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.162.89.153:16736/user/resourcemanager no longer participates in the leader election.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.747 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.747 [flink-akka.actor.default-dispatcher-43] INFO  org.apache.flink.runtime.dispatcher.MiniDispatcher  - Dispatcher akka.tcp://flink@10.162.89.153:16736/user/dispatcher was revoked leadership.
2019-10-09 13:54:21.747 [flink-akka.actor.default-dispatcher-46] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Suspending the SlotManager.
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:22.072 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.072 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:22.973 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.973 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.27/10.160.101.27:2181
2019-10-09 13:54:22.973 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.974 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:23.405 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:23.405 [main-SendThread(10.160.101.30:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.30/10.160.101.30:2181
2019-10-09 13:54:23.405 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:23.406 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.379 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.379 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:24.379 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.380 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x2597758ae5091ff for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: Connection refused
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:26.044 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-1334102078344377682.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:26.045 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:26.045 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:29.050 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.29/10.160.101.29:2181, initiating session
2019-10-09 13:54:29.095 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session establishment complete on server 10.160.101.29/10.160.101.29:2181, sessionid = 0x2597758ae5091ff, negotiated timeout = 40000
2019-10-09 13:54:29.095 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: RECONNECTED
2019-10-09 13:54:29.096 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:29.097 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:29.105 [flink-akka.actor.default-dispatcher-46] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.162.89.153:16736/user/resourcemanager was granted leadership with fencing token a22e4c9c80fd0c7249102f69c0b64e01
2019-10-09 13:54:29.105 [flink-akka.actor.default-dispatcher-46] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Starting the SlotManager.
2019-10-09 13:54:29.107 [main-EventThread] INFO  o.apache.flink.runtime.jobmaster.MiniDispatcherRestEndpoint  - http://10.162.89.153:12503 was granted leadership with leaderSessionID=53b02e00-95c3-4d34-a272-b84795b991a7
2019-10-09 13:54:29.116 [flink-akka.actor.default-dispatcher-26] INFO  org.apache.flink.runtime.dispatcher.MiniDispatcher  - Dispatcher akka.tcp://flink@10.162.89.153:16736/user/dispatcher was granted leadership with fencing token d645a3fe-8fee-453b-abd5-bd4cd82ebba1
2019-10-09 13:54:34.809 [flink-rest-server-netty-worker-thread-73] ERROR org.apache.flink.runtime.rest.handler.job.JobDetailsHandler  - Could not retrieve the redirect address.
java.util.concurrent.CompletionException: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:772)
	at akka.dispatch.OnComplete.internal(Future.scala:258)
	at akka.dispatch.OnComplete.internal(Future.scala:256)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
	at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
	at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
	at akka.actor.ActorRef.tell(ActorRef.scala:130)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendErrorIfSender(AkkaRpcActor.java:371)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:57)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)
	at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)
	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)
Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)
	... 14 common frames omitted
2019-10-09 13:54:56.007 [flink-rest-server-netty-worker-thread-74] ERROR org.apache.flink.runtime.rest.handler.job.JobDetailsHandler  - Could not retrieve the redirect address.
java.util.concurrent.CompletionException: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at java.util.concurrent.CompletableFuture.encodeThrowable(CompletableFuture.java:292)
	at java.util.concurrent.CompletableFuture.completeThrowable(CompletableFuture.java:308)
	at java.util.concurrent.CompletableFuture.uniApply(CompletableFuture.java:593)
	at java.util.concurrent.CompletableFuture$UniApply.tryFire(CompletableFuture.java:577)
	at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
	at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)
	at org.apache.flink.runtime.concurrent.FutureUtils$1.onComplete(FutureUtils.java:772)
	at akka.dispatch.OnComplete.internal(Future.scala:258)
	at akka.dispatch.OnComplete.internal(Future.scala:256)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:186)
	at akka.dispatch.japi$CallbackBridge.apply(Future.scala:183)
	at scala.concurrent.impl.CallbackRunnable.run(Promise.scala:32)
	at org.apache.flink.runtime.concurrent.Executors$DirectExecutionContext.execute(Executors.java:83)
	at scala.concurrent.impl.CallbackRunnable.executeWithValue(Promise.scala:40)
	at scala.concurrent.impl.Promise$DefaultPromise.tryComplete(Promise.scala:248)
	at akka.pattern.PromiseActorRef.$bang(AskSupport.scala:534)
	at akka.actor.ActorRef.tell(ActorRef.scala:130)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.sendErrorIfSender(AkkaRpcActor.java:371)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:57)
	at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)
	at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)
	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)
Caused by: org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(9f126af7b71d8b61dcb3687f3dd84166, LocalRpcInvocation(requestRestAddress(Time))) sent to akka.tcp://flink@10.162.89.153:16736/user/dispatcher because the fencing token is null.
	at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)
	... 14 common frames omitted

In the above log, the ha zk got reconnected after a suspend, but the job can not recover, and there existed a MiniDispatcherRestEndpoint.


for the job who can recover from the same condition, the log pasted as below:

2019-10-09 13:54:21.803 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: SUSPENDED
2019-10-09 13:54:21.804 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender http://10.160.140.216:52303 no longer participates in the leader election.
2019-10-09 13:54:21.804 [Curator-ConnectionStateManager-0] INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint  - http://10.160.140.216:52303 lost leadership
2019-10-09 13:54:21.804 [Curator-PathChildrenCache-0] WARN  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - ZooKeeper connection SUSPENDING. Changes to the submitted job graphs are not monitored (temporarily).
2019-10-09 13:54:21.805 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.805 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.160.140.216:26073/user/resourcemanager was revoked leadership. Clearing fencing token.
2019-10-09 13:54:21.805 [Curator-ConnectionStateManager-0] INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner  - JobManager for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) was revoked leadership at akka.tcp://flink@10.160.140.216:26073/user/jobmanager_2.
2019-10-09 13:54:21.805 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock.
2019-10-09 13:54:21.806 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/jobmanager_2 no longer participates in the leader election.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher was revoked leadership.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-80] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Stopping all currently running jobs of dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher.
2019-10-09 13:54:21.806 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/dispatcher no longer participates in the leader election.
2019-10-09 13:54:21.808 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) switched from state RUNNING to SUSPENDING.
org.apache.flink.util.FlinkException: JobManager is no longer the leader.
	at org.apache.flink.runtime.jobmaster.JobManagerRunner.revokeLeadership(JobManagerRunner.java:371)
	at org.apache.flink.runtime.leaderelection.ZooKeeperLeaderElectionService.notLeader(ZooKeeperLeaderElectionService.java:247)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$8.apply(LeaderLatch.java:640)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$8.apply(LeaderLatch.java:636)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93)
	at org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.setLeadership(LeaderLatch.java:635)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.handleStateChange(LeaderLatch.java:623)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch.access$000(LeaderLatch.java:64)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.recipes.leader.LeaderLatch$1.stateChanged(LeaderLatch.java:82)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:259)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$2.apply(ConnectionStateManager.java:255)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer$1.run(ListenerContainer.java:93)
	at org.apache.flink.shaded.curator.org.apache.curator.shaded.com.google.common.util.concurrent.MoreExecutors$SameThreadExecutorService.execute(MoreExecutors.java:297)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.listen.ListenerContainer.forEach(ListenerContainer.java:85)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager.processEvents(ConnectionStateManager.java:253)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager.access$000(ConnectionStateManager.java:43)
	at org.apache.flink.shaded.curator.org.apache.curator.framework.state.ConnectionStateManager$1.call(ConnectionStateManager.java:111)
	at java.util.concurrent.FutureTask.run(FutureTask.java:266)
	at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
	at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
	at java.lang.Thread.run(Thread.java:745)
2019-10-09 13:54:21.806 [flink-akka.actor.default-dispatcher-88] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Suspending the SlotManager.
2019-10-09 13:54:21.808 [Curator-ConnectionStateManager-0] WARN  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper suspended. Can no longer retrieve the leader from ZooKeeper.
2019-10-09 13:54:21.809 [Curator-ConnectionStateManager-0] WARN  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper suspended. The contender akka.tcp://flink@10.160.140.216:26073/user/resourcemanager no longer participates in the leader election.
2019-10-09 13:54:21.819 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (1/8) (4a825def14a88297d19c9e5738dbf668) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (2/8) (6517a467aaf7be56bfb3821b29852e20) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (3/8) (7a7ee5b1f3613b97d858564dbfaeb066) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.820 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (4/8) (2d878a7c86e9d6e0bb0fe737a42037e5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.821 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (5/8) (82b545cdf0918a7d14b8c595884a5caf) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.821 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (6/8) (fb178b7aed87e2fc7c8be11838575cb6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (7/8) (7d2956af9eca83b125a5f2e3d65c7454) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (8/8) (8673406c095fbb33ff63b8116c2dc6c4) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (1/60) (fde9e604f69f27dd1e1afc2d6f96d6a6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.823 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (2/60) (44429fa0423a6c4e2cfa73b7caf98e7f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (3/60) (ccb886b9e7f7f3e9391a66a77a95d936) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (4/60) (9dfbcca1af51c7565d082ae1d4fbca99) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (5/60) (984560fc276a70ffb3566ec5236a57d6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (6/60) (42c8e83ec209f1947dfc53a3451ca002) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.824 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (7/60) (2fff02384094c4889f8d583db6c7c495) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (8/60) (bd81e085522ef4915a9413517dcb51ec) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (9/60) (11762d1b367d402bca99be0624d637c6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (10/60) (bcba40c174b64e16b1a48e802e0251c8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (11/60) (5655a9923af5f1d40436e2531ca87ff9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (12/60) (b9e3017ec604802c151939077f6f4f46) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.825 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (13/60) (4fee14cb460433aa6178fe98315631ff) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (14/60) (ec80588d10b6a11cc52625f95ebf4858) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (15/60) (5d55b130964c3c164df9938bb9b9b8e9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (16/60) (b67388f779efeb6ffcae57aaaf0e992f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (17/60) (c91c36736380adc280b7b9235bd26609) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.826 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (18/60) (f658b30a9c2b4dee63de0fb4797d0c3b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (19/60) (bb92a17d4bd6279729ac6e9274ca80ee) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (20/60) (ecf8a4563990fba29df91ee769e5ea2e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (21/60) (8ea7047b457ad4bded12ea4dcea1576b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (22/60) (a2cea7a46ecffbd5275d355a29bc09f9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.827 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (23/60) (105456dc1d11fc9d8164ea0d54794d90) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (24/60) (faa255028f7bba0ef8b832616a09e0cc) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (25/60) (80a6454bca3392e1050b02c2b689edae) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (26/60) (de3561bf14ed9617a4727491ad959219) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.828 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (27/60) (585101b822efc77260c0f0a52eb5f856) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (28/60) (3205706648aa73f9e34d29c22428ff90) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (29/60) (24dd134cb75696784bade727d6c57459) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (30/60) (7d21e0c3934c4bfdc5349f0627fdd923) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (31/60) (07672677de03585cc630e1014143bec8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.829 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (32/60) (72159fe698efbaacfb4690b22e4dad03) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (33/60) (a2f3dce63aac51c82060f02646019af5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (34/60) (c5e4f81b8c2590d0f00c7a5c3c05f7a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (35/60) (87579d8547b643a5e1418dca333b426f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.830 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (36/60) (021f60ef60686370a2e9e6d92fa033ad) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (37/60) (65b2647fabcbfb1b23aaea63d4627509) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (38/60) (3f054f2dc5161bd67bbec03d711dea18) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (39/60) (9b7f93456479e1943de63da8ff8c88d3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.837 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (40/60) (f2c204e511ed878b8bbd9fded928c215) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (41/60) (e139c0820ecd8cb8c8fe168b9a847a1e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (42/60) (42b961618e0fe859f0761e50e0569113) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (43/60) (29928821d6f94e715cc2ab444880fb7b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (44/60) (83e9b131ba8684474e13236b67916ac3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (45/60) (fe3e5b53d7fbc4e795822944317e85bb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.838 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (46/60) (d7617e20894c359f95cd363faaa3b117) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (47/60) (e7545083f44b6a8338750542943d67f0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (48/60) (0fc95f3ca9dafaeb8d90cd262de82440) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (49/60) (aa41f35f708625f08bde4b3513e3d769) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (50/60) (4d01c8d2543d86f2cdd89db1b81df2af) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.839 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (51/60) (6d5314d1d924da1437dfee4ff243ffb9) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (52/60) (fad4dc6e3ba4d0be2abbecb7a960c914) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (53/60) (0ccd9b09fc32f61b45031906a7c612b0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (54/60) (1fbc38dc9181a399bd30de4e4044e261) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.840 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (55/60) (2ea15174000a056b0d747d1089f6585f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (56/60) (28baca2d42222f0a0182c5c89e1717c6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (57/60) (e1abb8bce0113fb035f4a36271b73f17) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (58/60) (9f4a99aad3d396dbbffc75bc9b40f0b6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (59/60) (d58fd6d1d88118d0037c3c2e92176b69) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.841 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - ParsingBolt (60/60) (765e9de291878316b4c69000bcd0b2df) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (1/30) (782206df8a2ad0f7a2946eedf0cbc827) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (2/30) (db788aa1e9fdb0022bf9d5a7dc257235) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (3/30) (8567c954df44048c35632a52314d9660) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.842 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (4/30) (b8d2a0f72c717e2a06e5770751bf20e7) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (5/30) (d72b93dbfa363c38bd6b7451a39c7d67) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (6/30) (559d7f2c0bcb8255df8963b3b4329f66) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (7/30) (8124a163b7727ce53601ba21c7a49acc) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (8/30) (c151f0d09157158e3f449a847c41cc65) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (9/30) (86b2c2178b48d7bb74742036b6489260) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.843 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (10/30) (f716beb167fbe51c1db7798c1f1794d3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (11/30) (b72e2b66c4c3fa24a2e6d247210f9e34) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (12/30) (e63a7ed9a8fb828a1cda0e206fa803e1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (13/30) (7dcb47fd7e88bdc86ae45344d9779e99) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (14/30) (ccc164979332f39f374664d758345301) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (15/30) (b21287c5c6b9b89124a1cd6810eb2a21) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.844 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (16/30) (91071077d5e04266d5b6d9833cf2f9e4) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (17/30) (b5ec2ff0c6decca89ff76223c25b9d9b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (18/30) (0774604957ec0d2680b8673b98dbbe7b) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (19/30) (6e402ae9f4d0b8755c0c7a2ef67f2c1a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (20/30) (51493f15a4b7166aa390f854a597c4f6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (21/30) (e329d85d95a8a6a142b8157bd3b2eeee) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.845 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (22/30) (c17a752c676bd4fa1559b6641f959fc7) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (23/30) (94bfdd3b1bb3be163c951b41aa80f5a8) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (24/30) (f4a378189dd403de5d4d5a3e77415f9e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (25/30) (d16ff3ac286ab1fd32caa9f46fca5619) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.846 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (26/30) (090c3aaa6bb23261e74a7bf70060a36d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (27/30) (9aeebb1f3b6d51be5388674955953c52) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (28/30) (23c2fceab79f540a9d89e2e4f092a865) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (29/30) (e6bb884157821902ae36d036737efaa5) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - DeDuplicationBolt (30/30) (afca1c29fa6c50a49c88d59d3df459f6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.847 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (1/30) (9adb75badd9cb8afbb4251a33ef27171) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (2/30) (473835002699bfb37c5b88edafd219a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (3/30) (e7e030cb12de97e633cf473d7e627206) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (4/30) (0459e2f2ea079437f62769f2aee861d2) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (5/30) (ed428b67cd250fd6407a0b5243aab514) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.848 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (6/30) (4a54f08765d6c9faac31ace85a406819) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (7/30) (24ab50af8fdab9de5f0915479cedc50f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (8/30) (5f365913e8e69f0bb8fad4fcac0337ea) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (9/30) (d4ac35d28253d1dd096fa6bab4fafc8a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (10/30) (1ad00f44a299183f81f61ed829410b1d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (11/30) (98238896820682b892815dfa2f62e6c2) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.849 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (12/30) (6019139a62f395abb0d3bb0fa51b326e) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (13/30) (73f72dd701d042dacd95988521873013) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (14/30) (3109dae5918d772b64e7d8b28b128dfe) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (15/30) (10c0effec4da40afe144a59556882680) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (16/30) (722f04d080fed0df95787fc11a7c95d0) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (17/30) (07461d9203eedf28c0a8c893a631ca2f) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.850 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (18/30) (206c625688c12fa82cfc3bae7b60f24d) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (19/30) (6fd477a3d452423a09cd0ba0494a6deb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (20/30) (d0aa4c46dbb46db09c9395e39b41a4c1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (21/30) (0dde7060b76feb5e7333e20d3fe4e7a1) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (22/30) (ce6749c3e5b996df8647603e4e8fbfdb) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (23/30) (8270b98412ef6613859f9ae0109850e3) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.851 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (24/30) (136806c81715dd66a3d5cf8f49ceb031) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (25/30) (e95b22dff10204581a86a35461bf5768) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (26/30) (9fff9a9c688f9e383f18661923a814ba) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (27/30) (2626c8dad0fbab32fadd995930211d8a) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (28/30) (70aa1e618f77758c684cae15b7722752) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.852 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (29/30) (ef3392ed8088af2c05212e0fb0a5aac6) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.853 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - StatisticsBolt (30/30) (6af3938acd0f635d95c9c432edeab539) switched from RUNNING to CANCELING.
2019-10-09 13:54:21.855 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Close ResourceManager connection 490c7cf7fc01baf7e0ece283d48690a4: JobManager is no longer the leader..
2019-10-09 13:54:21.855 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Suspending SlotPool.
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-80] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Stopping the JobMaster for job hdp-shoujiserver_CommercialDotingAccess(2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.856 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcActor  - The rpc endpoint org.apache.flink.runtime.jobmaster.slotpool.SlotPool has not been started yet. Discarding message org.apache.flink.runtime.rpc.messages.LocalRpcInvocation until processing is started.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Stopping SlotPool.
2019-10-09 13:54:21.857 [flink-akka.actor.default-dispatcher-20] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Stopping ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock'}.
2019-10-09 13:54:22.602 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.603 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:22.603 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.603 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:22.921 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:22.922 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:22.922 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:22.922 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.008 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.008 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:24.008 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.009 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:24.778 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:24.778 [main-SendThread(10.160.101.30:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.30/10.160.101.30:2181
2019-10-09 13:54:24.778 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:24.779 [main-SendThread(10.160.101.30:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session 0x4597758a3589ca6 for server null, unexpected error, closing socket connection and attempting reconnect
java.net.ConnectException: 拒绝连接
	at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method)
	at sun.nio.ch.SocketChannelImpl.finishConnect(SocketChannelImpl.java:717)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxnSocketNIO.doTransport(ClientCnxnSocketNIO.java:361)
	at org.apache.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn$SendThread.run(ClientCnxn.java:1141)
2019-10-09 13:54:26.748 [main-SendThread(10.160.101.27:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:26.748 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.27/10.160.101.27:2181
2019-10-09 13:54:26.749 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:26.749 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.27/10.160.101.27:2181, initiating session
2019-10-09 13:54:26.751 [main-SendThread(10.160.101.27:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:27.673 [main-SendThread(10.160.101.29:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:27.674 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.29/10.160.101.29:2181
2019-10-09 13:54:27.674 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:27.674 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.29/10.160.101.29:2181, initiating session
2019-10-09 13:54:27.675 [main-SendThread(10.160.101.29:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:27.865 [main-SendThread(10.160.101.28:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:27.865 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.28/10.160.101.28:2181
2019-10-09 13:54:27.865 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:27.866 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.28/10.160.101.28:2181, initiating session
2019-10-09 13:54:27.867 [main-SendThread(10.160.101.28:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Unable to read additional data from server sessionid 0x4597758a3589ca6, likely server has closed socket, closing socket connection and attempting reconnect
2019-10-09 13:54:28.078 [main-SendThread(10.160.101.26:2181)] WARN  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - SASL configuration failed: javax.security.auth.login.LoginException: No JAAS configuration section named 'Client' was found in specified JAAS configuration file: '/tmp/jaas-4748011049181233444.conf'. Will continue connection to Zookeeper server without SASL authentication, if Zookeeper server allows it.
2019-10-09 13:54:28.078 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening socket connection to server 10.160.101.26/10.160.101.26:2181
2019-10-09 13:54:28.079 [main-EventThread] ERROR o.a.flink.shaded.curator.org.apache.curator.ConnectionState  - Authentication failed
2019-10-09 13:54:28.079 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket connection established to 10.160.101.26/10.160.101.26:2181, initiating session
2019-10-09 13:54:28.080 [main-SendThread(10.160.101.26:2181)] INFO  o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session establishment complete on server 10.160.101.26/10.160.101.26:2181, sessionid = 0x4597758a3589ca6, negotiated timeout = 40000
2019-10-09 13:54:28.081 [main-EventThread] INFO  o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State change: RECONNECTED
2019-10-09 13:54:28.081 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.082 [Curator-PathChildrenCache-0] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - ZooKeeper connection RECONNECTED. Changes to the submitted job graphs are monitored again.
2019-10-09 13:54:28.082 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:28.082 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.083 [Curator-ConnectionStateManager-0] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Connection to ZooKeeper was reconnected. Leader retrieval can be restarted.
2019-10-09 13:54:28.083 [Curator-ConnectionStateManager-0] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Connection to ZooKeeper was reconnected. Leader election can be restarted.
2019-10-09 13:54:28.087 [flink-akka.actor.default-dispatcher-88] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Released locks of job graph 2657b25ff073a91c888ff78a28bd2331 from ZooKeeper.
2019-10-09 13:54:28.088 [main-EventThread] INFO  org.apache.flink.runtime.dispatcher.DispatcherRestEndpoint  - http://10.160.140.216:52303 was granted leadership with leaderSessionID=d80d351c-56ca-490f-aa78-681847d58cfe
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-83] INFO  org.apache.flink.yarn.YarnResourceManager  - ResourceManager akka.tcp://flink@10.160.140.216:26073/user/resourcemanager was granted leadership with fencing token 8b328120280016029994e898e40749f8
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-83] INFO  o.a.flink.runtime.resourcemanager.slotmanager.SlotManager  - Starting the SlotManager.
2019-10-09 13:54:28.092 [flink-akka.actor.default-dispatcher-48] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Dispatcher akka.tcp://flink@10.160.140.216:26073/user/dispatcher was granted leadership with fencing token 65d868d3-0665-4555-ae75-86975bb3c7a9
2019-10-09 13:54:28.093 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Recovering all persisted jobs.
2019-10-09 13:54:28.130 [flink-akka.actor.default-dispatcher-72] INFO  o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Recovered SubmittedJobGraph(2657b25ff073a91c888ff78a28bd2331, null).
2019-10-09 13:54:28.212 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000003 (akka.tcp://flink@10.160.167.236:37058/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:28.515 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000005 (akka.tcp://flink@10.162.92.119:34318/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.072 [flink-akka.actor.default-dispatcher-72] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000008 (akka.tcp://flink@10.160.132.173:48920/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.473 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  - Starting RPC endpoint for org.apache.flink.runtime.jobmaster.JobMaster at akka://flink/user/jobmanager_3 .
2019-10-09 13:54:29.474 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Initializing job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.487 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Using restart strategy FailureRateRestartStrategy(failuresInterval=3600000 msdelayInterval=30000 msmaxFailuresPerInterval=3) for hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.489 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.rpc.akka.AkkaRpcService  - Starting RPC endpoint for org.apache.flink.runtime.jobmaster.slotpool.SlotPool at akka://flink/user/25b2e77f-fe0c-4e2d-af38-8900a36077d7 .
2019-10-09 13:54:29.489 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job recovers via failover strategy: full graph restart
2019-10-09 13:54:29.490 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Running initialization on master for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331).
2019-10-09 13:54:29.490 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Successfully ran initialization on master in 0 ms.
2019-10-09 13:54:29.497 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Initialized in '/checkpoints/2657b25ff073a91c888ff78a28bd2331'.
2019-10-09 13:54:29.498 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Loading state backend via factory org.apache.flink.contrib.streaming.state.RocksDBStateBackendFactory
2019-10-09 13:54:29.498 [flink-akka.actor.default-dispatcher-88] WARN  org.apache.flink.configuration.Configuration  - Config uses deprecated configuration key 'state.backend.rocksdb.checkpointdir' instead of proper key 'state.backend.rocksdb.localdir'
2019-10-09 13:54:29.502 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Recovering checkpoints from ZooKeeper.
2019-10-09 13:54:29.513 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Found 5 checkpoints in ZooKeeper.
2019-10-09 13:54:29.514 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to fetch 5 checkpoints from storage.
2019-10-09 13:54:29.514 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33036.
2019-10-09 13:54:29.516 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.yarn.YarnResourceManager  - Registering TaskManager with ResourceID container_e64_1564717237973_1572220_01_000010 (akka.tcp://flink@10.160.124.173:30267/user/taskmanager_0) at ResourceManager
2019-10-09 13:54:29.523 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33037.
2019-10-09 13:54:29.532 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33038.
2019-10-09 13:54:29.540 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33039.
2019-10-09 13:54:29.549 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Trying to retrieve checkpoint 33040.
2019-10-09 13:54:30.712 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - Restoring job 2657b25ff073a91c888ff78a28bd2331 from latest valid checkpoint: Checkpoint 33040 @ 1570600391503 for 2657b25ff073a91c888ff78a28bd2331.
2019-10-09 13:54:30.715 [flink-akka.actor.default-dispatcher-88] INFO  org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - No master state to restore
2019-10-09 13:54:30.715 [flink-akka.actor.default-dispatcher-88] INFO  o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Starting ZooKeeperLeaderElectionService ZooKeeperLeaderElectionService{leaderPath='/leader/2657b25ff073a91c888ff78a28bd2331/job_manager_lock'}.
2019-10-09 13:54:30.721 [main-EventThread] INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner  - JobManager runner for job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) was granted leadership with session id 829a1a30-be5a-4e4a-aa0c-21cc71b45599 at akka.tcp://flink@10.160.140.216:26073/user/jobmanager_3.
2019-10-09 13:54:30.724 [flink-akka.actor.default-dispatcher-21] INFO  o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
2019-10-09 13:54:30.724 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.jobmaster.JobMaster  - Starting execution of job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331)
2019-10-09 13:54:30.725 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Job hdp-shoujiserver_CommercialDotingAccess (2657b25ff073a91c888ff78a28bd2331) switched from state CREATED to RUNNING.
2019-10-09 13:54:30.725 [flink-akka.actor.default-dispatcher-21] INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph  - Source: WeiShiDataAnalysisSpout (1/8) (d072a1bc3f3dbf7896ef619951d91c88) switched from CREATED to SCHEDULED.
After the zk reconnected after a suspend, the job was recovered, and you can see, there was a StandaloneDispatcher created.


Don't know why the 'FencingTokenException: Fencing token not set' exception happened, and don't know why different dispatchers were created.

Thanks

Joshua


On Wed, Oct 2, 2019 at 9:10 PM Till Rohrmann <[hidden email]> wrote:
Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

Cheers,
Till

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:
Hi Bruce,

I haven't seen such an exception yet, but maybe Till (in CC) can help.

Best,
Fabian

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 

Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Hanson, Bruce
In reply to this post by Till Rohrmann

Hi Till and Fabian,

 

My apologies for taking a week to reply; it took some time to reproduce the issue with debug logging. I’ve attached logs from a two minute period when the problem happened. I’m just sending this to you two to avoid sending the log file all over the place. If you’d like to have our conversation in the user group mailing list, that’s fine.

 

The job was submitted by using the job manager REST api starting at 20:33:46.262 and finishing at 20:34:01.547. This worked normally, and the job started running. We then run a monitor that polls the /overview endpoint of the JM REST api. This started polling at 20:34:31.380 and resulted in the JM throwing the FencingTokenException at 20:34:31:393, and the JM returned a 500 to our monitor. This will happen every time we poll until the monitor times out and then we tear down the cluster, even though the job is running, we can’t tell that it is. This is somewhat rare, happening maybe 5% of the time.

 

We’re running Flink 1.7.1. This issue only happens when we run in Job Manager High Availability mode. We provision two Job Managers, a 3-node zookeeper cluster, task managers and our monitor all in their own Kubernetes namespace. I can send you Zookeeper logs too if that would be helpful.

 

Thanks in advance for any help you can provide!

 

-Bruce

-- 

 

 

From: Till Rohrmann <[hidden email]>
Date: Wednesday, October 2, 2019 at 6:10 AM
To: Fabian Hueske <[hidden email]>
Cc: "Hanson, Bruce" <[hidden email]>, "[hidden email]" <[hidden email]>
Subject: Re: Fencing token exceptions from Job Manager High Availability mode

 

Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

 

Cheers,

Till

 

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:

Hi Bruce,

 

I haven't seen such an exception yet, but maybe Till (in CC) can help.

 

Best,

Fabian

 

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 


flink4.log.gz (227K) Download Attachment
Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Joshua Fan
Hi Till

After got your advice, I checked the log again. It seems not wholely the same as the condition you mentioned.

I would like to summarize the story in the belowed log.

Once a time, the zk connection  was not stable, so there happened 3 times suspended-reconnected.

After the first suspended-reconnected, the Minidispatcher tried to recover all jobs.

Then the second suspended-reconnected came, after this reconnected, there happened a 'The heartbeat of JobManager with id dbad79e0173c5658b029fba4d70e8084 timed out', and in this turn, the Minidispatcher didn't try to recover the job.

Due to the zk connection did not recover, the third suspended-reconnected came, after the zk reconnected, the Minidispatcher did not try to recover job ,but just repeated throw FencingTokenException, the AM was hanging, our alarm-system just
found the job was gone, but can not get a final state of the job. And the FencingTokenException was ongoing for nearly one day long before we killed the AM.

the whole log is attached.

Thanks

Joshua

On Fri, Oct 11, 2019 at 10:35 AM Hanson, Bruce <[hidden email]> wrote:

Hi Till and Fabian,

 

My apologies for taking a week to reply; it took some time to reproduce the issue with debug logging. I’ve attached logs from a two minute period when the problem happened. I’m just sending this to you two to avoid sending the log file all over the place. If you’d like to have our conversation in the user group mailing list, that’s fine.

 

The job was submitted by using the job manager REST api starting at 20:33:46.262 and finishing at 20:34:01.547. This worked normally, and the job started running. We then run a monitor that polls the /overview endpoint of the JM REST api. This started polling at 20:34:31.380 and resulted in the JM throwing the FencingTokenException at 20:34:31:393, and the JM returned a 500 to our monitor. This will happen every time we poll until the monitor times out and then we tear down the cluster, even though the job is running, we can’t tell that it is. This is somewhat rare, happening maybe 5% of the time.

 

We’re running Flink 1.7.1. This issue only happens when we run in Job Manager High Availability mode. We provision two Job Managers, a 3-node zookeeper cluster, task managers and our monitor all in their own Kubernetes namespace. I can send you Zookeeper logs too if that would be helpful.

 

Thanks in advance for any help you can provide!

 

-Bruce

-- 

 

 

From: Till Rohrmann <[hidden email]>
Date: Wednesday, October 2, 2019 at 6:10 AM
To: Fabian Hueske <[hidden email]>
Cc: "Hanson, Bruce" <[hidden email]>, "[hidden email]" <[hidden email]>
Subject: Re: Fencing token exceptions from Job Manager High Availability mode

 

Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

 

Cheers,

Till

 

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:

Hi Bruce,

 

I haven't seen such an exception yet, but maybe Till (in CC) can help.

 

Best,

Fabian

 

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 


can not recover from zk suspened.7z (70K) Download Attachment
Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Joshua Fan
Sorry to forget the version, we run flink 1.7 on yarn in a ha mode.

On Fri, Oct 11, 2019 at 12:02 PM Joshua Fan <[hidden email]> wrote:
Hi Till

After got your advice, I checked the log again. It seems not wholely the same as the condition you mentioned.

I would like to summarize the story in the belowed log.

Once a time, the zk connection  was not stable, so there happened 3 times suspended-reconnected.

After the first suspended-reconnected, the Minidispatcher tried to recover all jobs.

Then the second suspended-reconnected came, after this reconnected, there happened a 'The heartbeat of JobManager with id dbad79e0173c5658b029fba4d70e8084 timed out', and in this turn, the Minidispatcher didn't try to recover the job.

Due to the zk connection did not recover, the third suspended-reconnected came, after the zk reconnected, the Minidispatcher did not try to recover job ,but just repeated throw FencingTokenException, the AM was hanging, our alarm-system just
found the job was gone, but can not get a final state of the job. And the FencingTokenException was ongoing for nearly one day long before we killed the AM.

the whole log is attached.

Thanks

Joshua

On Fri, Oct 11, 2019 at 10:35 AM Hanson, Bruce <[hidden email]> wrote:

Hi Till and Fabian,

 

My apologies for taking a week to reply; it took some time to reproduce the issue with debug logging. I’ve attached logs from a two minute period when the problem happened. I’m just sending this to you two to avoid sending the log file all over the place. If you’d like to have our conversation in the user group mailing list, that’s fine.

 

The job was submitted by using the job manager REST api starting at 20:33:46.262 and finishing at 20:34:01.547. This worked normally, and the job started running. We then run a monitor that polls the /overview endpoint of the JM REST api. This started polling at 20:34:31.380 and resulted in the JM throwing the FencingTokenException at 20:34:31:393, and the JM returned a 500 to our monitor. This will happen every time we poll until the monitor times out and then we tear down the cluster, even though the job is running, we can’t tell that it is. This is somewhat rare, happening maybe 5% of the time.

 

We’re running Flink 1.7.1. This issue only happens when we run in Job Manager High Availability mode. We provision two Job Managers, a 3-node zookeeper cluster, task managers and our monitor all in their own Kubernetes namespace. I can send you Zookeeper logs too if that would be helpful.

 

Thanks in advance for any help you can provide!

 

-Bruce

-- 

 

 

From: Till Rohrmann <[hidden email]>
Date: Wednesday, October 2, 2019 at 6:10 AM
To: Fabian Hueske <[hidden email]>
Cc: "Hanson, Bruce" <[hidden email]>, "[hidden email]" <[hidden email]>
Subject: Re: Fencing token exceptions from Job Manager High Availability mode

 

Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

 

Cheers,

Till

 

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:

Hi Bruce,

 

I haven't seen such an exception yet, but maybe Till (in CC) can help.

 

Best,

Fabian

 

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0

 

 

Reply | Threaded
Open this post in threaded view
|

Re: Fencing token exceptions from Job Manager High Availability mode

Till Rohrmann
Hi,

[hidden email], I think the logs you provided me substantiated my suspicion. I think you are running into FLINK-11843. It happens the following way: After regaining the leadership, the Dispatcher tries to start the JobManager. Recovering the state of the job takes a while (could also be the temporary connection loss to ZooKeeper) so that the second revocation of the leadership happens during the checkpoint recovery. Due to this loss, the start of the JobManager cannot be completed. However, we need this operation to complete in order to start the second job recovery after regaining the leadership for the second time. This problem will be fixed with FLINK-11843. This problem will be fixed with Flink 1.10. The fix should be merged next week so that you could try out whether this problem still occurs then.

[hidden email], the problem seems to be the following: You start two JobManagers. Technically this means that two ClusterEntrypoints will be started. In each ClusterEntrypoint a Dispatcher, ResourceManager, RestServerEndpoint and potentially multiple JobManagers can be executed. In order to figure out which component is active we do leader election. In your current setup, the Dispatcher1 (running in the first ClusterEntrypoint process) and ResourceManager2 (running in the second ClusterEntrypoint process) gain leadership. Since Dispatcher1 only can talk to ResourceManager1, the cluster does not accept job submissions and the Dispatcher cannot serve the cluster overview. The problem has been fixed with Flink 1.8.0. Hence, I would suggest to upgrade to a newer Flink version where the problem should no longer occur. 

Cheers,
Till

On Fri, Oct 11, 2019 at 6:18 AM Joshua Fan <[hidden email]> wrote:
Sorry to forget the version, we run flink 1.7 on yarn in a ha mode.

On Fri, Oct 11, 2019 at 12:02 PM Joshua Fan <[hidden email]> wrote:
Hi Till

After got your advice, I checked the log again. It seems not wholely the same as the condition you mentioned.

I would like to summarize the story in the belowed log.

Once a time, the zk connection  was not stable, so there happened 3 times suspended-reconnected.

After the first suspended-reconnected, the Minidispatcher tried to recover all jobs.

Then the second suspended-reconnected came, after this reconnected, there happened a 'The heartbeat of JobManager with id dbad79e0173c5658b029fba4d70e8084 timed out', and in this turn, the Minidispatcher didn't try to recover the job.

Due to the zk connection did not recover, the third suspended-reconnected came, after the zk reconnected, the Minidispatcher did not try to recover job ,but just repeated throw FencingTokenException, the AM was hanging, our alarm-system just
found the job was gone, but can not get a final state of the job. And the FencingTokenException was ongoing for nearly one day long before we killed the AM.

the whole log is attached.

Thanks

Joshua

On Fri, Oct 11, 2019 at 10:35 AM Hanson, Bruce <[hidden email]> wrote:

Hi Till and Fabian,

 

My apologies for taking a week to reply; it took some time to reproduce the issue with debug logging. I’ve attached logs from a two minute period when the problem happened. I’m just sending this to you two to avoid sending the log file all over the place. If you’d like to have our conversation in the user group mailing list, that’s fine.

 

The job was submitted by using the job manager REST api starting at 20:33:46.262 and finishing at 20:34:01.547. This worked normally, and the job started running. We then run a monitor that polls the /overview endpoint of the JM REST api. This started polling at 20:34:31.380 and resulted in the JM throwing the FencingTokenException at 20:34:31:393, and the JM returned a 500 to our monitor. This will happen every time we poll until the monitor times out and then we tear down the cluster, even though the job is running, we can’t tell that it is. This is somewhat rare, happening maybe 5% of the time.

 

We’re running Flink 1.7.1. This issue only happens when we run in Job Manager High Availability mode. We provision two Job Managers, a 3-node zookeeper cluster, task managers and our monitor all in their own Kubernetes namespace. I can send you Zookeeper logs too if that would be helpful.

 

Thanks in advance for any help you can provide!

 

-Bruce

-- 

 

 

From: Till Rohrmann <[hidden email]>
Date: Wednesday, October 2, 2019 at 6:10 AM
To: Fabian Hueske <[hidden email]>
Cc: "Hanson, Bruce" <[hidden email]>, "[hidden email]" <[hidden email]>
Subject: Re: Fencing token exceptions from Job Manager High Availability mode

 

Hi Bruce, are you able to provide us with the full debug logs? From the excerpt itself it is hard to tell what is going on.

 

Cheers,

Till

 

On Wed, Oct 2, 2019 at 2:24 PM Fabian Hueske <[hidden email]> wrote:

Hi Bruce,

 

I haven't seen such an exception yet, but maybe Till (in CC) can help.

 

Best,

Fabian

 

Am Di., 1. Okt. 2019 um 05:51 Uhr schrieb Hanson, Bruce <[hidden email]>:

Hi all,

 

We are running some of our Flink jobs with Job Manager High Availability. Occasionally we get a cluster that comes up improperly and doesn’t respond. Attempts to submit the job seem to hang and when we hit the /overview REST endpoint in the Job Manager we get a 500 error and a fencing token exception like this:

 

2019-09-21 05:04:07.785 [flink-akka.actor.default-dispatcher-4428] level=ERROR o.a.f.runtime.rest.handler.cluster.ClusterOverviewHandler  - Implementation error: Unhandled exception.

org.apache.flink.runtime.rpc.exceptions.FencingTokenException: Fencing token not set: Ignoring message LocalFencedMessage(null, LocalRpcInvocation(requestResourceOverview(Time))) sent to akka.tcp://[hidden email]:6126/user/resourcemanager because the fencing token is null.

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:59)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.onReceive(FencedAkkaRpcActor.java:40)

        at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

        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)

 

 

We are running Flink 1.7.1 in Kubernetes and run each job in its own namespace with a three-node Zookeeper cluster and two Job Managers, plus one or more Task Managers. I have been able to replicate the issue, but don’t find any difference in the logs between a failing cluster and a good one.

 

Does anyone here have any ideas as to what’s happening, or what I should be looking for?

 

-Bruce

 

 

 

 

cid:image001.png@01D2B473.0F7F85E0

 

Bruce Hanson

Principal Engineer

M: +1 425 681 0422

 

HERE Technologies

701 Pike Street, Suite 2000

Seattle, WA 98101 USA

47° 36' 41" N 122° 19' 57" W

 

cid:image002.png@01D2B473.0F7F85E0    cid:image003.png@01D2B473.0F7F85E0   cid:image004.png@01D2B473.0F7F85E0    cid:image005.png@01D2B473.0F7F85E0    cid:image006.png@01D2B473.0F7F85E0