Hi Till,

Sorry for the late reply, I was waiting to update to Flink 1.6.0 to see if
the problem got fixed but I still experience the first issue (jobgraph not
deleted from zookeeper when task is canceled). The second issue
(taskmanagers unable to register to the new elected jobmanager) was
actually a configuration error, all jobmanagers had the
"jobmanager.rpc.address" option set to point to the same jobmanager so I
guess the new one was not registering its url correctly in Zookeeper.

And for the first issue, it seems to be a known bug:
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Old-job-resurrected-during-HA-failover-td22000.html
http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/JobGraphs-not-cleaned-up-in-HA-mode-td22600.html
https://issues.apache.org/jira/browse/FLINK-10011

so I guess I'll have to wait until there is a fix as I have not seens any
workaround (other than removing the jobgraph from Zookeeper after
cancelling the task)

Thanks,

Gerard


On Tue, Jul 24, 2018 at 1:46 PM Till Rohrmann <trohrm...@apache.org> wrote:

> Hi Gerard,
>
> the first log snippet from the client does not show anything suspicious.
> The warning just says that you cannot use the Yarn CLI because it lacks the
> Hadoop dependencies in the classpath.
>
> The second snippet is indeed more interesting. If the TaskExecutors are
> not notified about the changed leader, then this might indicate a problem
> with the ZooKeeper connection or the ZooKeeper cluster itself. This might
> also explain why the job deletion from ZooKeeper does not succeed.
>
> One thing you could check is whether the leader ZNode under
> `/flink/default/leader/dispatcher_lock` (if you are using the defaults)
> actually contains the address of the newly elected leader. The leader path
> should also be logged in the cluster entrypoint logs. You can use the
> ZooKeeper cli for accessing the ZNodes.
>
> Cheers,
> Till
>
> On Mon, Jul 23, 2018 at 4:07 PM Gerard Garcia <ger...@talaia.io> wrote:
>
>> We have just started experiencing a different problem that could be
>> related, maybe it helps to diagnose the issue.
>>
>> In the last 24h the jobmanager lost connection to Zookeeper a couple of
>> times. Each time, a new jobmanager (in a different node) was elected leader
>> correctly but the taskamangers kept trying to connect to the old
>> jobmanager. These are the ending log messages until the taskamanger shut
>> down itself.
>>
>> 12:06:41.747 [flink-akka.actor.default-dispatcher-5] WARN
>> akka.remote.transport.netty.NettyTransport New I/O boss #3 - Remote
>> connection to [null] failed with java.net.ConnectException: Connection
>> refused: (...)1/192.168.1.9:35605
>> 12:06:41.748 [flink-akka.actor.default-dispatcher-2] INFO
>> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Could not resolve
>> ResourceManager address akka.tcp://flink@(...)1:35605/user/resourcemanager,
>> retrying in 10000 ms: Could not connect to rpc endpoint under address
>> akka.tcp://flink@(...)1:35605/user/resourcemanager..
>> 12:06:41.748 [flink-akka.actor.default-dispatcher-5] WARN
>> akka.remote.ReliableDeliverySupervisor
>> flink-akka.remote.default-remote-dispatcher-15 - Association with remote
>> system [akka.tcp://flink@(...)1:35605] has failed, address is now gated
>> for [50] ms. Reason: [Association failed with 
>> [akka.tcp://flink@(...)1:35605]]
>> Caused by: [Connection refused: (...)1/192.168.1.9:35605]
>> 12:06:51.766 [flink-akka.actor.default-dispatcher-5] WARN
>> akka.remote.transport.netty.NettyTransport New I/O boss #3 - Remote
>> connection to [null] failed with java.net.ConnectException: Connection
>> refused: (...)1/192.168.1.9:35605
>> 12:06:51.767 [flink-akka.actor.default-dispatcher-2] INFO
>> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Could not resolve
>> ResourceManager address akka.tcp://flink@(...)1:35605/user/resourcemanager,
>> retrying in 10000 ms: Could not connect to rpc endpoint under address
>> akka.tcp://flink@(...)1:35605/user/resourcemanager..
>> 12:06:51.767 [flink-akka.actor.default-dispatcher-5] WARN
>> akka.remote.ReliableDeliverySupervisor
>> flink-akka.remote.default-remote-dispatcher-7 - Association with remote
>> system [akka.tcp://flink@(...)1:35605] has failed, address is now gated
>> for [50] ms. Reason: [Association failed with 
>> [akka.tcp://flink@(...)1:35605]]
>> Caused by: [Connection refused: (...)1/192.168.1.9:35605]
>> 12:07:01.123 [flink-akka.actor.default-dispatcher-5] ERROR
>> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Fatal error occurred
>> in TaskExecutor akka.tcp://flink@(...)2:33455/user/taskmanager_0.
>> org.apache.flink.runtime.taskexecutor.exceptions.RegistrationTimeoutException:
>> Could not register at the ResourceManager within the specified maximum
>> registration duration 300000 ms. This indicates a problem with this
>> instance. Terminating now.
>>         at
>> org.apache.flink.runtime.taskexecutor.TaskExecutor.registrationTimeout(TaskExecutor.java:1018)
>>         at
>> org.apache.flink.runtime.taskexecutor.TaskExecutor.lambda$startRegistrationTimeout$3(TaskExecutor.java:1004)
>>         at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:332)
>>         at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:158)
>>         at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
>>         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)
>> 12:07:01.125 [flink-akka.actor.default-dispatcher-5] ERROR
>> org.apache.flink.runtime.taskexecutor.TaskManagerRunner  - Fatal error
>> occurred while executing the TaskManager. Shutting it down...
>> org.apache.flink.runtime.taskexecutor.exceptions.RegistrationTimeoutException:
>> Could not register at the ResourceManager within the specified maximum
>> registration duration 300000 ms. This indicates a problem with this
>> instance. Terminating now.
>>         at
>> org.apache.flink.runtime.taskexecutor.TaskExecutor.registrationTimeout(TaskExecutor.java:1018)
>>         at
>> org.apache.flink.runtime.taskexecutor.TaskExecutor.lambda$startRegistrationTimeout$3(TaskExecutor.java:1004)
>>         at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:332)
>>         at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:158)
>>         at
>> org.apache.flink.runtime.rpc.akka.AkkaRpcActor.onReceive(AkkaRpcActor.java:142)
>>         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)
>> 12:07:01.128 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Stopping TaskExecutor
>> akka.tcp://flink@(...)2:33455/user/taskmanager_0.
>> 12:07:01.128 [flink-akka.actor.default-dispatcher-5] INFO
>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping
>> ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>> 12:07:01.128 [flink-akka.actor.default-dispatcher-5] INFO
>> o.a.flink.runtime.state.TaskExecutorLocalStateStoresManager  - Shutting
>> down TaskExecutorLocalStateStoresManager.
>> 12:07:01.130 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.io.disk.iomanager.IOManager  - I/O manager removed
>> spill file directory /home/tmp/flink-io-cf83b38e-53f1-4802-a097-e6db95b46084
>> 12:07:01.130 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.io.network.NetworkEnvironment  - Shutting down the
>> network environment and its components.
>> 12:07:01.131 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.io.network.netty.NettyClient  - Successful
>> shutdown (took 1 ms).
>> 12:07:01.132 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.io.network.netty.NettyServer  - Successful
>> shutdown (took 1 ms).
>> 12:07:01.141 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.taskexecutor.JobLeaderService  - Stop job leader
>> service.
>> 12:07:01.142 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.taskexecutor.TaskExecutor  - Stopped TaskExecutor
>> akka.tcp://flink@(...)2:33455/user/taskmanager_0.
>> 12:07:01.143 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.blob.PermanentBlobCache  - Shutting down BLOB cache
>> 12:07:01.143 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.blob.TransientBlobCache  - Shutting down BLOB cache
>> 12:07:01.148 [Curator-Framework-0] INFO
>> o.a.f.s.c.o.a.curator.framework.imps.CuratorFrameworkImpl  -
>> backgroundOperationsLoop exiting
>> 12:07:01.173 [flink-akka.actor.default-dispatcher-5] INFO
>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Session:
>> 0x3011955028f00f4 closed
>> 12:07:01.173 [main-EventThread] INFO
>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - EventThread
>> shut down for session: 0x3011955028f00f4
>> 12:07:01.173 [flink-akka.actor.default-dispatcher-5] INFO
>> org.apache.flink.runtime.rpc.akka.AkkaRpcService  - Stopping Akka RPC
>> service.
>> 12:07:01.178 [flink-akka.actor.default-dispatcher-3] INFO
>> akka.remote.RemoteActorRefProvider$RemotingTerminator
>> flink-akka.remote.default-remote-dispatcher-16 - Shutting down remote
>> daemon.
>> 12:07:01.178 [flink-akka.actor.default-dispatcher-3] INFO
>> akka.remote.RemoteActorRefProvider$RemotingTerminator
>> flink-akka.remote.default-remote-dispatcher-16 - Remote daemon shut down;
>> proceeding with flushing remote transports.
>> 12:07:01.190 [flink-akka.actor.default-dispatcher-5] INFO
>> akka.remote.RemoteActorRefProvider$RemotingTerminator
>> flink-akka.remote.default-remote-dispatcher-7 - Remoting shut down.
>>
>> It seems like there are problems updating Zookeeper. I've have also
>> noticed these messages in Zookeeper log:
>>
>>  WARN  [SyncThread:2:FileTxnLog@378] - fsync-ing the write ahead log in
>> SyncThread:2 took 1259ms which will adversely effect operation latency. See
>> the ZooKeeper troubleshooting guide
>>
>> Maybe Flink hits some timeout?
>>
>> Gerard
>>
>> On Mon, Jul 23, 2018 at 11:57 AM Gerard Garcia <ger...@talaia.io> wrote:
>>
>>> Hi Till,
>>>
>>> I can't post the full log (as there is internal info in them) but I've
>>> found this. Is that what you are looking for?
>>>
>>> 11:29:17.351 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> --------------------------------------------------------------------------------
>>> 11:29:17.372 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Starting Command Line Client (Version: 1.5-SNAPSHOT, Rev:a4fc4c6,
>>> Date:05.06.2018 @ 10:22:30 CEST)
>>> 11:29:17.372 [main] INFO  org.apache.flink.client.cli.CliFrontend  -  OS
>>> current user: (...)
>>> 11:29:17.372 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Current Hadoop/Kerberos user: <no hadoop dependency found>
>>> 11:29:17.372 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> JVM: Java HotSpot(TM) 64-Bit Server VM - Oracle Corporation - 1.8/25.131-b11
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Maximum heap size: 14254 MiBytes
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> JAVA_HOME: (not set)
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -  No
>>> Hadoop Dependency available
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> JVM Options:
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>>  -Dlog.file=/opt/flink/flink-1.5.0/log/flink-root-client-(...).log
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>>  -Dlog4j.configuration=file:/opt/flink/flink-1.5.0/conf/log4j-cli.properties
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>>  -Dlogback.configurationFile=file:/opt/flink/flink-1.5.0/conf/logback.xml
>>> 11:29:17.373 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Program Arguments:
>>> 11:29:17.374 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>>  cancel
>>> 11:29:17.374 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>>  e403893e5208ca47ace886a77e405291
>>> 11:29:17.374 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Classpath:
>>> /opt/flink/flink-1.5.0/lib/commons-httpclient-3.1.jar:/opt/flink/flink-1.5.0/lib/flink-metrics-statsd-1.5.0.jar:/opt/flink/flink-1.5.0/lib/flink-python_2.11-1.5.0.jar:/opt/flink/flink-1.5.0/lib/fluency-1.8.0.jar:/opt/flink/flink-1.5.0/lib/gcs-connector-latest-hadoop2.jar:/opt/flink/flink-1.5.0/lib/hadoop-openstack-2.7.1.jar:/opt/flink/flink-1.5.0/lib/jackson-annotations-2.8.0.jar:/opt/flink/flink-1.5.0/lib/jackson-core-2.8.10.jar:/opt/flink/flink-1.5.0/lib/jackson-databind-2.8.11.1.jar:/opt/flink/flink-1.5.0/lib/jackson-dataformat-msgpack-0.8.15.jar:/opt/flink/flink-1.5.0/lib/log4j-1.2.17.jar:/opt/flink/flink-1.5.0/lib/log4j-over-slf4j-1.7.25.jar:/opt/flink/flink-1.5.0/lib/logback-classic-1.2.3.jar:/opt/flink/flink-1.5.0/lib/logback-core-1.2.3.jar:/opt/flink/flink-1.5.0/lib/logback-more-appenders-1.4.2.jar:/opt/flink/flink-1.5.0/lib/msgpack-0.6.12.jar:/opt/flink/flink-1.5.0/lib/msgpack-core-0.8.15.jar:/opt/flink/flink-1.5.0/lib/phi-accural-failure-detector-0.0.4.jar:/opt/flink/flink-1.5.0/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/flink-1.5.0/lib/flink-dist_2.11-1.5.0.jar:::
>>> 11:29:17.375 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> --------------------------------------------------------------------------------
>>> 11:29:17.380 [main] WARN  org.apache.flink.client.cli.CliFrontend  -
>>> Could not load CLI class org.apache.flink.yarn.cli.FlinkYarnSessionCli.
>>> java.lang.NoClassDefFoundError: org/apache/hadoop/conf/Configuration
>>>         at java.lang.Class.forName0(Native Method)
>>>         at java.lang.Class.forName(Class.java:264)
>>>         at
>>> org.apache.flink.client.cli.CliFrontend.loadCustomCommandLine(CliFrontend.java:1204)
>>>         at
>>> org.apache.flink.client.cli.CliFrontend.loadCustomCommandLines(CliFrontend.java:1160)
>>>         at
>>> org.apache.flink.client.cli.CliFrontend.main(CliFrontend.java:1086)
>>> Caused by: java.lang.ClassNotFoundException:
>>> org.apache.hadoop.conf.Configuration
>>>         at java.net.URLClassLoader.findClass(URLClassLoader.java:381)
>>>         at java.lang.ClassLoader.loadClass(ClassLoader.java:424)
>>>         at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:335)
>>>         at java.lang.ClassLoader.loadClass(ClassLoader.java:357)
>>>         ... 5 common frames omitted
>>> 11:29:17.385 [main] INFO  org.apache.flink.core.fs.FileSystem  - Hadoop
>>> is not in the classpath/dependencies. The extended set of supported File
>>> Systems via Hadoop is not available.
>>> 11:29:17.479 [main] INFO
>>> o.apache.flink.runtime.security.modules.HadoopModuleFactory  - Cannot
>>> create Hadoop Security Module because Hadoop cannot be found in the
>>> Classpath.
>>> 11:29:17.489 [main] INFO
>>> org.apache.flink.runtime.security.SecurityUtils  - Cannot install
>>> HadoopSecurityContext because Hadoop cannot be found in the Classpath.
>>> 11:29:17.518 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Running 'cancel' command.
>>> 11:29:17.523 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Cancelling job e403893e5208ca47ace886a77e405291.
>>> 11:29:17.537 [main] INFO
>>> org.apache.flink.runtime.blob.FileSystemBlobStore  - Creating highly
>>> available BLOB storage directory at file:///home/nas/flink/ha//default/blob
>>> 11:29:17.538 [main] INFO  org.apache.flink.runtime.util.ZooKeeperUtils
>>> - Enforcing default ACL for ZK connections
>>> 11:29:17.539 [main] INFO  org.apache.flink.runtime.util.ZooKeeperUtils
>>> - Using '/flink/default' as Zookeeper namespace.
>>> 11:29:17.574 [main] INFO
>>> o.a.f.s.c.o.a.curator.framework.imps.CuratorFrameworkImpl  - Starting
>>> 11:29:17.577 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:zookeeper.version=3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f,
>>> built on 03/23/2017 10:13 GMT
>>> 11:29:17.577 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:host.name=(...)
>>> 11:29:17.578 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.version=1.8.0_131
>>> 11:29:17.578 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.vendor=Oracle Corporation
>>> 11:29:17.579 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.home=/opt/jdk/jdk1.8.0_131/jre
>>> 11:29:17.580 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.class.path=/opt/flink/flink-1.5.0/lib/commons-httpclient-3.1.jar:/opt/flink/flink-1.5.0/lib/flink-metrics-statsd-1.5.0.jar:/opt/flink/flink-1.5.0/lib/flink-python_2.11-1.5.0.jar:/opt/flink/flink-1.5.0/lib/fluency-1.8.0.jar:/opt/flink/flink-1.5.0/lib/gcs-connector-latest-hadoop2.jar:/opt/flink/flink-1.5.0/lib/hadoop-openstack-2.7.1.jar:/opt/flink/flink-1.5.0/lib/jackson-annotations-2.8.0.jar:/opt/flink/flink-1.5.0/lib/jackson-core-2.8.10.jar:/opt/flink/flink-1.5.0/lib/jackson-databind-2.8.11.1.jar:/opt/flink/flink-1.5.0/lib/jackson-dataformat-msgpack-0.8.15.jar:/opt/flink/flink-1.5.0/lib/log4j-1.2.17.jar:/opt/flink/flink-1.5.0/lib/log4j-over-slf4j-1.7.25.jar:/opt/flink/flink-1.5.0/lib/logback-classic-1.2.3.jar:/opt/flink/flink-1.5.0/lib/logback-core-1.2.3.jar:/opt/flink/flink-1.5.0/lib/logback-more-appenders-1.4.2.jar:/opt/flink/flink-1.5.0/lib/msgpack-0.6.12.jar:/opt/flink/flink-1.5.0/lib/msgpack-core-0.8.15.jar:/opt/flink/flink-1.5.0/lib/phi-accural-failure-detector-0.0.4.jar:/opt/flink/flink-1.5.0/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/flink-1.5.0/lib/flink-dist_2.11-1.5.0.jar:::
>>> 11:29:17.580 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib
>>> 11:29:17.580 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.io.tmpdir=/tmp
>>> 11:29:17.580 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:java.compiler=<NA>
>>> 11:29:17.580 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.name=Linux
>>> 11:29:17.580 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.arch=amd64
>>> 11:29:17.580 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:os.version=4.9.87-xxxx-std-ipv6-64
>>> 11:29:17.581 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.name=(...)
>>> 11:29:17.581 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.home=(...)
>>> 11:29:17.581 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>> environment:user.dir=/opt/flink/flink-1.5.0/bin
>>> 11:29:17.581 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Initiating
>>> client connection, connectString=10.1.1.5:2181,10.1.1.6:2181,
>>> 10.1.1.7:2181 sessionTimeout=60000
>>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@4a003cbe
>>> 11:29:17.589 [main-SendThread(10.1.1.5: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-3807415919448894740.conf'. Will continue
>>> connection to Zookeeper server without SASL authentication, if Zookeeper
>>> server allows it.
>>> 11:29:17.590 [main-SendThread(10.1.1.5:2181)] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening
>>> socket connection to server 10.1.1.5/10.1.1.5:2181
>>> 11:29:17.590 [main-EventThread] ERROR
>>> o.a.flink.shaded.curator.org.apache.curator.ConnectionState  -
>>> Authentication failed
>>> 11:29:17.603 [main-SendThread(10.1.1.5:2181)] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>>> connection established to 10.1.1.5/10.1.1.5:2181, initiating session
>>> 11:29:17.625 [main-SendThread(10.1.1.5:2181)] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session
>>> establishment complete on server 10.1.1.5/10.1.1.5:2181, sessionid =
>>> 0x100571bda1903c3, negotiated timeout = 40000
>>> 11:29:17.626 [main-EventThread] INFO
>>> o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State
>>> change: CONNECTED
>>> 11:29:17.764 [main] INFO  org.apache.flink.runtime.rest.RestClient  -
>>> Rest client endpoint started.
>>> 11:29:17.766 [main] INFO
>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting
>>> ZooKeeperLeaderRetrievalService /leader/rest_server_lock.
>>> 11:29:17.812 [main] INFO
>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting
>>> ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>> 11:29:18.007 [main] INFO  org.apache.flink.runtime.rest.RestClient  -
>>> Shutting down rest endpoint.
>>> 11:29:18.008 [main] INFO  org.apache.flink.runtime.rest.RestClient  -
>>> Rest endpoint shutdown complete.
>>> 11:29:18.008 [main] INFO
>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping
>>> ZooKeeperLeaderRetrievalService /leader/rest_server_lock.
>>> 11:29:18.009 [main] INFO
>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping
>>> ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>> 11:29:18.010 [Curator-Framework-0] INFO
>>> o.a.f.s.c.o.a.curator.framework.imps.CuratorFrameworkImpl  -
>>> backgroundOperationsLoop exiting
>>> 11:29:18.030 [main] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Session:
>>> 0x100571bda1903c3 closed
>>> 11:29:18.030 [main-EventThread] INFO
>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - EventThread
>>> shut down for session: 0x100571bda1903c3
>>> 11:29:18.030 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>> Cancelled job e403893e5208ca47ace886a77e405291.
>>>
>>> Gerard
>>>
>>> On Fri, Jul 20, 2018 at 5:14 AM vino yang <yanghua1...@gmail.com> wrote:
>>>
>>>> Hi Till,
>>>>
>>>> You are right, we also saw the problem you said. Curator removes the
>>>> specific job graph path asynchronously. But it's the only gist when
>>>> recovering, right? Is there any plan to enhance this point?
>>>>
>>>> Thanks, vino.
>>>>
>>>> 2018-07-19 21:58 GMT+08:00 Till Rohrmann <trohrm...@apache.org>:
>>>>
>>>>> Hi Gerard,
>>>>>
>>>>> the logging statement `Removed job graph ... from ZooKeeper` is
>>>>> actually not 100% accurate. The actual deletion is executed as an
>>>>> asynchronous background task and the log statement is not printed in the
>>>>> callback (which it should). Therefore, the deletion could still have
>>>>> failed. In order to see this, the full jobmanager/cluster entry point logs
>>>>> would be tremendously helpful.
>>>>>
>>>>> Cheers,
>>>>> Till
>>>>>
>>>>> On Thu, Jul 19, 2018 at 1:33 PM Gerard Garcia <ger...@talaia.io>
>>>>> wrote:
>>>>>
>>>>>> Thanks Andrey,
>>>>>>
>>>>>> That is the log from the jobmanager just after it has finished
>>>>>> cancelling the task:
>>>>>>
>>>>>> 11:29:18.716 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - Stopping
>>>>>> checkpoint coordinator for job e403893e5208ca47ace886a77e405291.
>>>>>> 11:29:18.716 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Shutting 
>>>>>> down
>>>>>> 11:29:18.738 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> o.a.f.runtime.checkpoint.ZooKeeperCompletedCheckpointStore  - Removing
>>>>>> /flink-eur/default/checkpoints/e403893e5208ca47ace886a77e405291 from
>>>>>> ZooKeeper
>>>>>> 11:29:18.780 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> o.a.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter  - Shutting 
>>>>>> down.
>>>>>> 11:29:18.780 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> o.a.flink.runtime.checkpoint.ZooKeeperCheckpointIDCounter  - Removing
>>>>>> /checkpoint-counter/e403893e5208ca47ace886a77e405291 from ZooKeeper
>>>>>> 11:29:18.827 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher  - Job
>>>>>> e403893e5208ca47ace886a77e405291 reached globally terminal state 
>>>>>> CANCELED.
>>>>>> 11:29:18.846 [flink-akka.actor.default-dispatcher-15675] INFO
>>>>>> org.apache.flink.runtime.jobmaster.JobMaster  - Stopping the JobMaster 
>>>>>> for
>>>>>> job (...)(e403893e5208ca47ace886a77e405291).
>>>>>> 11:29:18.848 [flink-akka.actor.default-dispatcher-15675] INFO
>>>>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping
>>>>>> ZooKeeperLeaderRetrievalService /leader/resource_manager_lock.
>>>>>> 11:29:18.864 [flink-akka.actor.default-dispatcher-15675] INFO
>>>>>> org.apache.flink.runtime.jobmaster.JobMaster  - Close ResourceManager
>>>>>> connection d5fbc30a895066054e29fb2fd60fb0f1: JobManager is shutting 
>>>>>> down..
>>>>>> 11:29:18.864 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Suspending 
>>>>>> SlotPool.
>>>>>> 11:29:18.864 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> org.apache.flink.runtime.jobmaster.slotpool.SlotPool  - Stopping 
>>>>>> SlotPool.
>>>>>> 11:29:18.864 [flink-akka.actor.default-dispatcher-15688] INFO
>>>>>> o.a.flink.runtime.resourcemanager.StandaloneResourceManager  - Disconnect
>>>>>> job manager 
>>>>>> 9cf221e2340597629fb932c03aa14...@akka.tcp://flink@(...):33827/user/jobmanager_9
>>>>>> for job e403893e5208ca47ace886a77e405291 from the resource manager.
>>>>>> 11:29:18.864 [flink-akka.actor.default-dispatcher-15675] INFO
>>>>>> o.a.f.runtime.leaderelection.ZooKeeperLeaderElectionService  - Stopping
>>>>>> ZooKeeperLeaderElectionService
>>>>>> ZooKeeperLeaderElectionService{leaderPath='/leader/e403893e5208ca47ace886a77e405291/job_manager_lock'}.
>>>>>> 11:29:18.980 [flink-akka.actor.default-dispatcher-15695] INFO
>>>>>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator  - Completed
>>>>>> checkpoint 31154 for job 5d8c376b10d358b9c9470b3e70113626 (132520 bytes 
>>>>>> in
>>>>>> 411 ms).
>>>>>> 11:29:19.025 [flink-akka.actor.default-dispatcher-15683] INFO
>>>>>> o.a.flink.runtime.jobmanager.ZooKeeperSubmittedJobGraphStore  - Removed 
>>>>>> job
>>>>>> graph e403893e5208ca47ace886a77e405291 from ZooKeeper.
>>>>>>
>>>>>>
>>>>>> At the end it says removed job graph e403893e5208ca47ace886a77e405291
>>>>>> from ZooKeeper but I still can see it at /flink/default/jobgraphs:
>>>>>>
>>>>>> [zk: localhost:2181(CONNECTED) 14] ls
>>>>>> /flink/default/jobgraphs/e403893e5208ca47ace886a77e405291
>>>>>> [3fe9c3c8-5bec-404e-a720-75f9b188124f,
>>>>>> 36208299-0f6d-462c-bae4-2e3d53f50e8c]
>>>>>>
>>>>>> Gerard
>>>>>>
>>>>>> On Wed, Jul 18, 2018 at 4:24 PM Andrey Zagrebin <
>>>>>> and...@data-artisans.com> wrote:
>>>>>>
>>>>>>> Hi Gerard,
>>>>>>>
>>>>>>> There is an issue recently fixed for 1.5.2, 1.6.0:
>>>>>>> https://issues.apache.org/jira/browse/FLINK-9575
>>>>>>> It might have caused your problem.
>>>>>>>
>>>>>>> Can you please provide log from JobManager/Entry point for further
>>>>>>> investigation?
>>>>>>>
>>>>>>> Cheers,
>>>>>>> Andrey
>>>>>>>
>>>>>>> On 18 Jul 2018, at 10:16, Gerard Garcia <ger...@talaia.io> wrote:
>>>>>>>
>>>>>>> Hi vino,
>>>>>>>
>>>>>>> Seems that jobs id stay in /jobgraphs when we cancel them manually.
>>>>>>> For example, after cancelling the job with id
>>>>>>> 75e16686cb4fe0d33ead8e29af131d09 the entry is still in zookeeper's path
>>>>>>> /flink/default/jobgraphs, but the job disappeared from
>>>>>>> /home/nas/flink/ha/default/blob/.
>>>>>>>
>>>>>>> That is the client log:
>>>>>>>
>>>>>>> 09:20:58.492 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>>>>>> Cancelling job 75e16686cb4fe0d33ead8e29af131d09.
>>>>>>> 09:20:58.503 [main] INFO
>>>>>>> org.apache.flink.runtime.blob.FileSystemBlobStore  - Creating highly
>>>>>>> available BLOB storage directory at
>>>>>>> file:///home/nas/flink/ha//default/blob
>>>>>>> 09:20:58.505 [main] INFO
>>>>>>> org.apache.flink.runtime.util.ZooKeeperUtils  - Enforcing default ACL 
>>>>>>> for
>>>>>>> ZK connections
>>>>>>> 09:20:58.505 [main] INFO
>>>>>>> org.apache.flink.runtime.util.ZooKeeperUtils  - Using 
>>>>>>> '/flink-eur/default'
>>>>>>> as Zookeeper namespace.
>>>>>>> 09:20:58.539 [main] INFO
>>>>>>> o.a.f.s.c.o.a.curator.framework.imps.CuratorFrameworkImpl  - Starting
>>>>>>> 09:20:58.543 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:zookeeper.version=
>>>>>>> 3.4.10-39d3a4f269333c922ed3db283be479f9deacaa0f, built on 03/23/2017
>>>>>>> 10:13 GMT
>>>>>>> 09:20:58.543 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:host.name=flink-eur-production1
>>>>>>> 09:20:58.543 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:java.version=1.8.0_131
>>>>>>> 09:20:58.544 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:java.vendor=Oracle Corporation
>>>>>>> 09:20:58.546 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:java.home=/opt/jdk/jdk1.8.0_131/jre
>>>>>>> 09:20:58.546 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:java.class.path=/opt/flink/flink-1.5.0/lib/commons-httpclient-3.1.jar:/opt/flink/flink-1.5.0/lib/flink-metrics-statsd-1.5.0.jar:/opt/flink/flink-1.5.0/lib/flink-python_2.11-1.5.0.jar:/opt/flink/flink-1.5.0/lib/fluency-1.8.0.jar:/opt/flink/flink-1.5.0/lib/gcs-connector-latest-hadoop2.jar:/opt/flink/flink-1.5.0/lib/hadoop-openstack-2.7.1.jar:/opt/flink/flink-1.5.0/lib/jackson-annotations-2.8.0.jar:/opt/flink/flink-1.5.0/lib/jackson-core-2.8.10.jar:/opt/flink/flink-1.5.0/lib/jackson-databind-2.8.11.1.jar:/opt/flink/flink-1.5.0/lib/jackson-dataformat-msgpack-0.8.15.jar:/opt/flink/flink-1.5.0/lib/log4j-1.2.17.jar:/opt/flink/flink-1.5.0/lib/log4j-over-slf4j-1.7.25.jar:/opt/flink/flink-1.5.0/lib/logback-classic-1.2.3.jar:/opt/flink/flink-1.5.0/lib/logback-core-1.2.3.jar:/opt/flink/flink-1.5.0/lib/logback-more-appenders-1.4.2.jar:/opt/flink/flink-1.5.0/lib/msgpack-0.6.12.jar:/opt/flink/flink-1.5.0/lib/msgpack-core-0.8.15.jar:/opt/flink/flink-1.5.0/lib/phi-accural-failure-detector-0.0.4.jar:/opt/flink/flink-1.5.0/lib/slf4j-log4j12-1.7.7.jar:/opt/flink/flink-1.5.0/lib/flink-dist_2.11-1.5.0.jar:::
>>>>>>> 09:20:58.546 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:java.library.path=/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib
>>>>>>> 09:20:58.546 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:java.io.tmpdir=/tmp
>>>>>>> 09:20:58.546 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:java.compiler=<NA>
>>>>>>> 09:20:58.547 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:os.name=Linux
>>>>>>> 09:20:58.547 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:os.arch=amd64
>>>>>>> 09:20:58.547 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:os.version=4.9.87-xxxx-std-ipv6-64
>>>>>>> 09:20:58.547 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:user.name=root
>>>>>>> 09:20:58.547 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:user.home=/root
>>>>>>> 09:20:58.547 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Client
>>>>>>> environment:user.dir=/opt/flink/flink-1.5.0/bin
>>>>>>> 09:20:58.548 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Initiating
>>>>>>> client connection, connectString=10.1.1.5:2181,10.1.1.6:2181,
>>>>>>> 10.1.1.7:2181 sessionTimeout=60000
>>>>>>> watcher=org.apache.flink.shaded.curator.org.apache.curator.ConnectionState@4a003cbe
>>>>>>> 09:20:58.555 [main-SendThread(10.1.1.5: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-9143038863636945274.conf'. Will continue
>>>>>>> connection to Zookeeper server without SASL authentication, if Zookeeper
>>>>>>> server allows it.
>>>>>>> 09:20:58.556 [main-SendThread(10.1.1.5:2181)] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Opening
>>>>>>> socket connection to server 10.1.1.5/10.1.1.5:2181
>>>>>>> 09:20:58.556 [main-EventThread] ERROR 
>>>>>>> o.a.flink.shaded.curator.org.apache.curator.ConnectionState
>>>>>>> - Authentication failed
>>>>>>> 09:20:58.569 [main-SendThread(10.1.1.5:2181)] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Socket
>>>>>>> connection established to 10.1.1.5/10.1.1.5:2181, initiating session
>>>>>>> 09:20:58.592 [main-SendThread(10.1.1.5:2181)] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - Session
>>>>>>> establishment complete on server 10.1.1.5/10.1.1.5:2181, sessionid
>>>>>>> = 0x100571bda1903b7, negotiated timeout = 40000
>>>>>>> 09:20:58.593 [main-EventThread] INFO
>>>>>>> o.a.f.s.c.o.a.curator.framework.state.ConnectionStateManager  - State
>>>>>>> change: CONNECTED
>>>>>>> 09:20:58.711 [main] INFO  org.apache.flink.runtime.rest.RestClient
>>>>>>> - Rest client endpoint started.
>>>>>>> 09:20:58.713 [main] INFO
>>>>>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting
>>>>>>> ZooKeeperLeaderRetrievalService /leader/rest_server_lock.
>>>>>>> 09:20:58.755 [main] INFO
>>>>>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Starting
>>>>>>> ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>>>>>> 09:20:58.946 [main] INFO  org.apache.flink.runtime.rest.RestClient
>>>>>>> - Shutting down rest endpoint.
>>>>>>> 09:20:58.946 [main] INFO  org.apache.flink.runtime.rest.RestClient
>>>>>>> - Rest endpoint shutdown complete.
>>>>>>> 09:20:58.947 [main] INFO
>>>>>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping
>>>>>>> ZooKeeperLeaderRetrievalService /leader/rest_server_lock.
>>>>>>> 09:20:58.948 [main] INFO
>>>>>>> o.a.f.r.leaderretrieval.ZooKeeperLeaderRetrievalService  - Stopping
>>>>>>> ZooKeeperLeaderRetrievalService /leader/dispatcher_lock.
>>>>>>> 09:20:58.949 [Curator-Framework-0] INFO
>>>>>>> o.a.f.s.c.o.a.curator.framework.imps.CuratorFrameworkImpl  -
>>>>>>> backgroundOperationsLoop exiting
>>>>>>> 09:20:58.968 [main] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ZooKeeper  - Session:
>>>>>>> 0x100571bda1903b7 closed
>>>>>>> 09:20:58.968 [main] INFO  org.apache.flink.client.cli.CliFrontend  -
>>>>>>> Cancelled job 75e16686cb4fe0d33ead8e29af131d09.
>>>>>>> 09:20:58.969 [main-EventThread] INFO
>>>>>>> o.a.flink.shaded.zookeeper.org.apache.zookeeper.ClientCnxn  - 
>>>>>>> EventThread
>>>>>>> shut down for session: 0x100571bda1903b7
>>>>>>>
>>>>>>> I'm assuming that in /jobgraphs there should only be the job ids
>>>>>>> that are currently running (at least it seemed that when the jobmanager
>>>>>>> restarted it tried to restart the jobs ids stored there). Is that 
>>>>>>> correct?
>>>>>>>
>>>>>>> Gerard
>>>>>>>
>>>>>>> On Wed, Jul 18, 2018 at 9:17 AM vino yang <yanghua1...@gmail.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>>> Hi Gerard,
>>>>>>>>
>>>>>>>> From you provide information, you mean the path in Zookeeper
>>>>>>>> "/jobgraphs" exists more jobs than you submitted?
>>>>>>>> And can not be restarted because blob files can not be find?
>>>>>>>>
>>>>>>>> Can you provide more details, about the stack trace, log and which
>>>>>>>> version of Flink? Normally, the jobgraph can not be added to Zookeeper
>>>>>>>> except submit job manually.
>>>>>>>>
>>>>>>>> Thanks, vino.
>>>>>>>>
>>>>>>>> 2018-07-16 21:19 GMT+08:00 gerardg <ger...@talaia.io>:
>>>>>>>>
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> Our deployment consists of a standalone HA cluster of 8 machines
>>>>>>>>> with an
>>>>>>>>> external Zookeeper cluster. We have observed several times that
>>>>>>>>> when a
>>>>>>>>> jobmanager fails and a new one is elected, the new one tries to
>>>>>>>>> restart
>>>>>>>>> more jobs than the ones that were running and since it can't find
>>>>>>>>> some
>>>>>>>>> files, it fails and gets stuck in a restart loop. That is the
>>>>>>>>> error that we
>>>>>>>>> see in the logs:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> These are the contents of /home/nas/flink/ha/default/blob/:
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> We've checked zookeeper and there are actually a lot of jobgraphs
>>>>>>>>> in
>>>>>>>>> /flink/default/jobgraphs
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> There were only three jobs running so neither zookeeper nor the
>>>>>>>>> flink 'ha'
>>>>>>>>> folder seems to have the correct number of jobgraphs stored.
>>>>>>>>>
>>>>>>>>> The only way we have to solve this is to remove everything at path
>>>>>>>>> /flink in
>>>>>>>>> zookeeper and the 'ha' flink folder and restart the jobs manually.
>>>>>>>>>
>>>>>>>>> I'll try to monitor if some action (e.g. we have been canceling and
>>>>>>>>> restoring jobs from savepoints quite often lately) leaves an entry
>>>>>>>>> in
>>>>>>>>> zookeepers path /flink/default/jobgraphs of a job that is not
>>>>>>>>> running but
>>>>>>>>> maybe someone can't point us to some configuration problem that
>>>>>>>>> could cause
>>>>>>>>> this behavior.
>>>>>>>>>
>>>>>>>>> Thanks,
>>>>>>>>>
>>>>>>>>> Gerard
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>>
>>>>>>>>> --
>>>>>>>>> Sent from:
>>>>>>>>> http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/
>>>>>>>>>
>>>>>>>>
>>>>>>>>
>>>>>>>
>>>>

Reply via email to