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