Cool. I'm happy that you were able to validate the issue. I'll write a fix for it: https://issues.apache.org/jira/browse/FLINK-3712
On Thu, Apr 7, 2016 at 10:43 AM, Konstantin Knauf < konstantin.kn...@tngtech.com> wrote: > Hi everyone, > > thanks to Robert, I found the problem. > > I was setting "recovery.zookeeper.path.root" on the command line with > -yD. Apparently this is currently not supported. You need to set it the > parameter in flink-conf.yaml. > > Cheers, > > Konstantin > > On 05.04.2016 12:52, Konstantin Knauf wrote: > > Hi Robert, > > > > I tried several paths and rmr before. > > > > It stopped after 1-2 minutes. There was an exception on the shell. > > Sorry, should have attached to the last mail. > > > > Thanks, > > > > Konstnatin > > > > On 05.04.2016 11:22, Robert Metzger wrote: > >> I've tried reproducing the issue on a test cluster, but everything > >> worked fine. > >> > >> Have you tried different values for "recovery.zookeeper.path.root" or > >> only one? Maybe the path you've put contains invalid data? > >> > >> Regarding the client log you've send: Did you manually stop the client > >> or did it stop after a few minutes? > >> The JobManager stops after a few minutes because the client requested a > >> shutdown. Usually, the client only shuts down on an exception or when > >> the user stops the yarn session. > >> There is no exception in the client log. Was there an exception printed > >> to the shell? > >> > >> This log message: > >> > >> 2016-04-05 08:48:34,912 DEBUG org.apache.flink.yarn.FlinkYarnCluster > >> - Received message option None > >> > >> Should not be an issue. > >> > >> > >> On Tue, Apr 5, 2016 at 10:14 AM, Ufuk Celebi <u...@apache.org > >> <mailto:u...@apache.org>> wrote: > >> > >> Hey Konstantin, > >> > >> just looked at the logs and the cluster is started, but the job is > >> indeed never submitted. > >> > >> I've forwarded this to Robert, because he is familiar with the YARN > >> client. I will look into how the client interacts with the ZooKeeper > >> root path. > >> > >> – Ufuk > >> > >> > >> On Tue, Apr 5, 2016 at 9:18 AM, Konstantin Knauf > >> <konstantin.kn...@tngtech.com <mailto:konstantin.kn...@tngtech.com > >> > >> wrote: > >> > Hi Ufuk, Hi Stephan, > >> > > >> > sorry for the late response Attached the client logs. > >> > > >> > Cheers, > >> > > >> > Konstantin > >> > > >> > On 04.04.2016 21 <tel:04.04.2016%2021>:20, Stephan Ewen wrote: > >> >> This seems to the the critical part in the logs: > >> >> > >> >> 2016-03-31 09:01:52,234 INFO > org.apache.flink.yarn.YarnJobManager > >> >> - Re-submitting 0 job graphs. > >> >> 2016-03-31 09:02:51,182 INFO > org.apache.flink.yarn.YarnJobManager > >> >> - Stopping YARN JobManager with status FAILED > and > >> >> diagnostic Flink YARN Client requested shutdown. > >> >> > >> >> The YarnJobManager starts up properly, but the Client never sends > >> >> anything, shuts down at some point, and tears down the YARN > cluster. > >> >> > >> >> Client logs would help a lot there... > >> >> > >> >> > >> >> > >> >> > >> >> On Sat, Apr 2, 2016 at 12:43 PM, Ufuk Celebi <u...@apache.org > >> <mailto:u...@apache.org> > >> >> <mailto:u...@apache.org <mailto:u...@apache.org>>> wrote: > >> >> > >> >> Hey Konstantin, > >> >> > >> >> That's weird. Can you please log the client output on DEBUG > >> level and > >> >> provide that as well? I'm wondering whether the client uses a > >> >> different root path. > >> >> > >> >> The following seems to happen: > >> >> - you use ledf_recovery as the root namespace > >> >> - the task managers are connecting (they resolve the JM > >> address via > >> >> ZooKeeper in this case as well, which means they correctly > >> use the > >> >> same namespace) > >> >> - but the client, which started the YARN session, does not > >> ever submit > >> >> the job to the cluster. > >> >> > >> >> – Ufuk > >> >> > >> >> On Thu, Mar 31, 2016 at 9:23 AM, Konstantin Knauf > >> >> <konstantin.kn...@tngtech.com > >> <mailto:konstantin.kn...@tngtech.com> > >> <mailto:konstantin.kn...@tngtech.com > >> <mailto:konstantin.kn...@tngtech.com>>> > >> >> wrote: > >> >> > Hi everyone, > >> >> > > >> >> > we are running in some problems with multiple per-job yarn > >> >> sessions, too. > >> >> > > >> >> > When we are are starting a per-job yarn session (Flink 1.0, > >> Hadoop > >> >> 2.4) > >> >> > with recovery.zookeeper.path.root other than /flink, the > >> yarn session > >> >> > starts but no job is submitted, and after 1 min or so the > >> session > >> >> > crashes. I attached the jobmanager log. > >> >> > > >> >> > In Zookeeper the root-directory is created and child-nodes > >> >> > > >> >> > leaderlatch > >> >> > jobgraphs > >> >> > > >> >> > /flink does also exist, but does not have child nodes. > >> >> > > >> >> > Everything runs fine, with the default > >> recovery.zookeeper.root.path. > >> >> > > >> >> > Does anyone have an idea, what is going on? > >> >> > > >> >> > Cheers, > >> >> > > >> >> > Konstnatin > >> >> > > >> >> > > >> >> > On 23.11.2015 17:00, Gwenhael Pasquiers wrote: > >> >> >> We are not yet using HA in our cluster instances. > >> >> >> > >> >> >> But yes, we will have to change the zookeeper.path.root J > >> >> >> > >> >> >> > >> >> >> > >> >> >> We package our jobs with their own config folder (we don’t > >> rely on > >> >> >> flink’s config folder); we can put the maven project name > >> into this > >> >> >> property then they will have different values J > >> >> >> > >> >> >> > >> >> >> > >> >> >> > >> >> >> > >> >> >> *From:*Till Rohrmann [mailto:trohrm...@apache.org > >> <mailto:trohrm...@apache.org> > >> >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org>>] > >> >> >> *Sent:* lundi 23 novembre 2015 14:51 > >> >> >> *To:* user@flink.apache.org <mailto:user@flink.apache.org > > > >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org>> > >> >> >> *Subject:* Re: YARN High Availability > >> >> >> > >> >> >> > >> >> >> > >> >> >> The problem is the execution graph handle which is stored > in > >> >> ZooKeeper. > >> >> >> You can manually remove it via the ZooKeeper shell by > >> simply deleting > >> >> >> everything below your `recovery.zookeeper.path.root` > >> ZNode. But you > >> >> >> should be sure that the cluster has been stopped before. > >> >> >> > >> >> >> > >> >> >> > >> >> >> Do you start the different clusters with different > >> >> >> `recovery.zookeeper.path.root` values? If not, then you > should > >> >> run into > >> >> >> troubles when running multiple clusters at the same time. > The > >> >> reason is > >> >> >> that then all clusters will think that they belong > together. > >> >> >> > >> >> >> > >> >> >> > >> >> >> Cheers, > >> >> >> > >> >> >> Till > >> >> >> > >> >> >> > >> >> >> > >> >> >> On Mon, Nov 23, 2015 at 2:15 PM, Gwenhael Pasquiers > >> >> >> <gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>> > >> >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>>>> wrote: > >> >> >> > >> >> >> OK, I understand. > >> >> >> > >> >> >> Maybe we are not really using flink as you intended. The > >> way we are > >> >> >> using it, one cluster equals one job. That way we are sure > >> to isolate > >> >> >> the different jobs as much as possible and in case of > >> crashes / > >> >> bugs / > >> >> >> (etc) can completely kill one cluster without interfering > with > >> >> the other > >> >> >> jobs. > >> >> >> > >> >> >> That future behavior seems good :-) > >> >> >> > >> >> >> Instead of the manual flink commands, is there to manually > >> delete > >> >> those > >> >> >> old jobs before launching my job ? They probably are > >> somewhere in > >> >> hdfs, > >> >> >> aren't they ? > >> >> >> > >> >> >> B.R. > >> >> >> > >> >> >> > >> >> >> > >> >> >> -----Original Message----- > >> >> >> From: Ufuk Celebi [mailto:u...@apache.org > >> <mailto:u...@apache.org> <mailto:u...@apache.org <mailto: > u...@apache.org>> > >> >> <mailto:u...@apache.org <mailto:u...@apache.org> > >> <mailto:u...@apache.org <mailto:u...@apache.org>>>] > >> >> >> Sent: lundi 23 novembre 2015 12:12 > >> >> >> To: user@flink.apache.org <mailto:user@flink.apache.org> > >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org>> > >> >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org> > >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org>>> > >> >> >> Subject: Re: YARN High Availability > >> >> >> > >> >> >> Hey Gwenhaël, > >> >> >> > >> >> >> the restarting jobs are most likely old job submissions. > >> They are not > >> >> >> cleaned up when you shut down the cluster, but only when > >> they finish > >> >> >> (either regular finish or after cancelling). > >> >> >> > >> >> >> The workaround is to use the command line frontend: > >> >> >> > >> >> >> bin/flink cancel JOBID > >> >> >> > >> >> >> for each RESTARTING job. Sorry about the inconvenience! > >> >> >> > >> >> >> We are in an active discussion about addressing this. The > >> future > >> >> >> behaviour will be that the startup or shutdown of a > >> cluster cleans up > >> >> >> everything and an option to skip this step. > >> >> >> > >> >> >> The reasoning for the initial solution (not removing > >> anything) was to > >> >> >> make sure that no jobs are deleted by accident. But it > >> looks like > >> >> this > >> >> >> is more confusing than helpful. > >> >> >> > >> >> >> – Ufuk > >> >> >> > >> >> >>> On 23 Nov 2015, at 11:45, Gwenhael Pasquiers > >> >> >> <gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>> > >> >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>>>> wrote: > >> >> >>> > >> >> >>> Hi again ! > >> >> >>> > >> >> >>> On the same topic I'm still trying to start my streaming > job > >> >> with HA. > >> >> >>> The HA part seems to be more or less OK (I killed the > >> JobManager and > >> >> >> it came back), however I have an issue with the > TaskManagers. > >> >> >>> I configured my job to have only one TaskManager and 1 > >> slot that > >> >> does > >> >> >> [source=>map=>sink]. > >> >> >>> The issue I'm encountering is that other instances of my > >> job appear > >> >> >> and are in the RESTARTING status since there is only one > >> task slot. > >> >> >>> > >> >> >>> Do you know of this, or have an idea of where to look in > >> order to > >> >> >> understand what's happening ? > >> >> >>> > >> >> >>> B.R. > >> >> >>> > >> >> >>> Gwenhaël PASQUIERS > >> >> >>> > >> >> >>> -----Original Message----- > >> >> >>> From: Maximilian Michels [mailto:m...@apache.org > >> <mailto:m...@apache.org> > >> >> <mailto:m...@apache.org <mailto:m...@apache.org>> > >> <mailto:m...@apache.org <mailto:m...@apache.org> > >> <mailto:m...@apache.org <mailto:m...@apache.org>>>] > >> >> >>> Sent: jeudi 19 novembre 2015 13:36 > >> >> >>> To: user@flink.apache.org <mailto:user@flink.apache.org> > >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org>> > >> >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org> > >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org>>> > >> >> >>> Subject: Re: YARN High Availability > >> >> >>> > >> >> >>> The docs have been updated. > >> >> >>> > >> >> >>> On Thu, Nov 19, 2015 at 12:36 PM, Ufuk Celebi > >> <u...@apache.org <mailto:u...@apache.org> > >> >> <mailto:u...@apache.org <mailto:u...@apache.org>> > >> >> >> <mailto:u...@apache.org <mailto:u...@apache.org> > >> <mailto:u...@apache.org <mailto:u...@apache.org>>>> wrote: > >> >> >>>> I’ve added a note about this to the docs and asked Max > >> to trigger a > >> >> >> new build of them. > >> >> >>>> > >> >> >>>> Regarding Aljoscha’s idea: I like it. It is essentially > >> a shortcut > >> >> >> for configuring the root path. > >> >> >>>> > >> >> >>>> In any case, it is orthogonal to Till’s proposals. That > >> one we need > >> >> >> to address as well (see FLINK-2929). The motivation for > >> the current > >> >> >> behaviour was to be rather defensive when removing state > >> in order > >> >> to not > >> >> >> loose data accidentally. But it can be confusing, indeed. > >> >> >>>> > >> >> >>>> – Ufuk > >> >> >>>> > >> >> >>>>> On 19 Nov 2015, at 12:08, Till Rohrmann > >> <trohrm...@apache.org <mailto:trohrm...@apache.org> > >> >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org>> > >> >> >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org > > > >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org>>>> > wrote: > >> >> >>>>> > >> >> >>>>> You mean an additional start-up parameter for the > >> >> `start-cluster.sh` > >> >> >> script for the HA case? That could work. > >> >> >>>>> > >> >> >>>>> On Thu, Nov 19, 2015 at 11:54 AM, Aljoscha Krettek > >> >> >> <aljos...@apache.org <mailto:aljos...@apache.org> > >> <mailto:aljos...@apache.org <mailto:aljos...@apache.org>> > >> >> <mailto:aljos...@apache.org <mailto:aljos...@apache.org> > >> <mailto:aljos...@apache.org <mailto:aljos...@apache.org>>>> wrote: > >> >> >>>>> Maybe we could add a user parameter to specify a > >> cluster name that > >> >> >> is used to make the paths unique. > >> >> >>>>> > >> >> >>>>> > >> >> >>>>> On Thu, Nov 19, 2015, 11:24 Till Rohrmann > >> >> <trohrm...@apache.org <mailto:trohrm...@apache.org> > >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org>> > >> >> >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org > > > >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org>>>> > wrote: > >> >> >>>>> I agree that this would make the configuration easier. > >> However, it > >> >> >> entails also that the user has to retrieve the randomized > path > >> >> from the > >> >> >> logs if he wants to restart jobs after the cluster has > >> crashed or > >> >> >> intentionally restarted. Furthermore, the system won't be > >> able to > >> >> clean > >> >> >> up old checkpoint and job handles in case that the cluster > >> stop was > >> >> >> intentional. > >> >> >>>>> > >> >> >>>>> Thus, the question is how do we define the behaviour in > >> order to > >> >> >> retrieve handles and to clean up old handles so that > ZooKeeper > >> >> won't be > >> >> >> cluttered with old handles? > >> >> >>>>> > >> >> >>>>> There are basically two modes: > >> >> >>>>> > >> >> >>>>> 1. Keep state handles when shutting down the cluster. > >> Provide > >> >> a mean > >> >> >> to define a fixed path when starting the cluster and also > >> a mean to > >> >> >> purge old state handles. Furthermore, add a shutdown mode > >> where the > >> >> >> handles under the current path are directly removed. This > >> mode would > >> >> >> guarantee to always have the state handles available if > not > >> >> explicitly > >> >> >> told differently. However, the downside is that ZooKeeper > >> will be > >> >> >> cluttered most certainly. > >> >> >>>>> > >> >> >>>>> 2. Remove the state handles when shutting down the > cluster. > >> >> Provide > >> >> >> a shutdown mode where we keep the state handles. This will > >> keep > >> >> >> ZooKeeper clean but will give you also the possibility to > >> keep a > >> >> >> checkpoint around if necessary. However, the user is more > >> likely > >> >> to lose > >> >> >> his state when shutting down the cluster. > >> >> >>>>> > >> >> >>>>> On Thu, Nov 19, 2015 at 10:55 AM, Robert Metzger > >> >> >> <rmetz...@apache.org <mailto:rmetz...@apache.org> > >> <mailto:rmetz...@apache.org <mailto:rmetz...@apache.org>> > >> >> <mailto:rmetz...@apache.org <mailto:rmetz...@apache.org> > >> <mailto:rmetz...@apache.org <mailto:rmetz...@apache.org>>>> wrote: > >> >> >>>>> I agree with Aljoscha. Many companies install Flink > >> (and its > >> >> config) > >> >> >> in a central directory and users share that installation. > >> >> >>>>> > >> >> >>>>> On Thu, Nov 19, 2015 at 10:45 AM, Aljoscha Krettek > >> >> >> <aljos...@apache.org <mailto:aljos...@apache.org> > >> <mailto:aljos...@apache.org <mailto:aljos...@apache.org>> > >> >> <mailto:aljos...@apache.org <mailto:aljos...@apache.org> > >> <mailto:aljos...@apache.org <mailto:aljos...@apache.org>>>> wrote: > >> >> >>>>> I think we should find a way to randomize the paths > >> where the HA > >> >> >> stuff stores data. If users don’t realize that they store > >> data in the > >> >> >> same paths this could lead to problems. > >> >> >>>>> > >> >> >>>>>> On 19 Nov 2015, at 08:50, Till Rohrmann > >> <trohrm...@apache.org <mailto:trohrm...@apache.org> > >> >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org>> > >> >> >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org > > > >> <mailto:trohrm...@apache.org <mailto:trohrm...@apache.org>>>> > wrote: > >> >> >>>>>> > >> >> >>>>>> Hi Gwenhaël, > >> >> >>>>>> > >> >> >>>>>> good to hear that you could resolve the problem. > >> >> >>>>>> > >> >> >>>>>> When you run multiple HA flink jobs in the same > >> cluster, then you > >> >> >> don’t have to adjust the configuration of Flink. It should > >> work > >> >> out of > >> >> >> the box. > >> >> >>>>>> > >> >> >>>>>> However, if you run multiple HA Flink cluster, then > >> you have > >> >> to set > >> >> >> for each cluster a distinct ZooKeeper root path via the > option > >> >> >> recovery.zookeeper.path.root in the Flink configuraiton. > >> This is > >> >> >> necessary because otherwise all JobManagers (the ones of > the > >> >> different > >> >> >> clusters) will compete for a single leadership. > >> Furthermore, all > >> >> >> TaskManagers will only see the one and only leader and > >> connect to it. > >> >> >> The reason is that the TaskManagers will look up their > >> leader at > >> >> a ZNode > >> >> >> below the ZooKeeper root path. > >> >> >>>>>> > >> >> >>>>>> If you have other questions then don’t hesitate > asking me. > >> >> >>>>>> > >> >> >>>>>> Cheers, > >> >> >>>>>> Till > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> On Wed, Nov 18, 2015 at 6:37 PM, Gwenhael Pasquiers > >> >> >> <gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>> > >> >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>>>> wrote: > >> >> >>>>>> Nevermind, > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Looking at the logs I saw that it was having issues > >> trying to > >> >> >> connect to ZK. > >> >> >>>>>> > >> >> >>>>>> To make I short is had the wrong port. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> It is now starting. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Tomorrow I’ll try to kill some JobManagers *evil*. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Another question : if I have multiple HA flink jobs, > are > >> >> there some > >> >> >> points to check in order to be sure that they won’t > collide on > >> >> hdfs or ZK ? > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> B.R. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Gwenhaël PASQUIERS > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> From: Till Rohrmann [mailto:till.rohrm...@gmail.com > >> <mailto:till.rohrm...@gmail.com> > >> >> <mailto:till.rohrm...@gmail.com <mailto: > till.rohrm...@gmail.com>> > >> >> >> <mailto:till.rohrm...@gmail.com > >> <mailto:till.rohrm...@gmail.com> <mailto:till.rohrm...@gmail.com > >> <mailto:till.rohrm...@gmail.com>>>] > >> >> >>>>>> Sent: mercredi 18 novembre 2015 18:01 > >> >> >>>>>> To: user@flink.apache.org > >> <mailto:user@flink.apache.org> <mailto:user@flink.apache.org > >> <mailto:user@flink.apache.org>> > >> >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org> > >> <mailto:user@flink.apache.org <mailto:user@flink.apache.org>>> > >> >> >>>>>> Subject: Re: YARN High Availability > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Hi Gwenhaël, > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> do you have access to the yarn logs? > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Cheers, > >> >> >>>>>> > >> >> >>>>>> Till > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> On Wed, Nov 18, 2015 at 5:55 PM, Gwenhael Pasquiers > >> >> >> <gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>> > >> >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com> > >> >> <mailto:gwenhael.pasqui...@ericsson.com > >> <mailto:gwenhael.pasqui...@ericsson.com>>>> wrote: > >> >> >>>>>> > >> >> >>>>>> Hello, > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> We’re trying to set up high availability using an > existing > >> >> >> zookeeper quorum already running in our Cloudera cluster. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> So, as per the doc we’ve changed the max attempt in > >> yarn’s config > >> >> >> as well as the flink.yaml. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> recovery.mode: zookeeper > >> >> >>>>>> > >> >> >>>>>> recovery.zookeeper.quorum: > >> host1:3181,host2:3181,host3:3181 > >> >> >>>>>> > >> >> >>>>>> state.backend: filesystem > >> >> >>>>>> > >> >> >>>>>> state.backend.fs.checkpointdir: > hdfs:///flink/checkpoints > >> >> >>>>>> > >> >> >>>>>> recovery.zookeeper.storageDir: hdfs:///flink/recovery/ > >> >> >>>>>> > >> >> >>>>>> yarn.application-attempts: 1000 > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Everything is ok as long as recovery.mode is > commented. > >> >> >>>>>> > >> >> >>>>>> As soon as I uncomment recovery.mode the deployment on > >> yarn is > >> >> >> stuck on : > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> “Deploying cluster, current state ACCEPTED”. > >> >> >>>>>> > >> >> >>>>>> “Deployment took more than 60 seconds….” > >> >> >>>>>> > >> >> >>>>>> Every second. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> And I have more than enough resources available on my > yarn > >> >> cluster. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Do you have any idea of what could cause this, and/or > >> what logs I > >> >> >> should look for in order to understand ? > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> B.R. > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> Gwenhaël PASQUIERS > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>>> > >> >> >>>>> > >> >> >>>>> > >> >> >>>>> > >> >> >>>>> > >> >> >>>> > >> >> >>> <unwanted_jobs.jpg> > >> >> >> > >> >> >> > >> >> >> > >> >> > > >> >> > -- > >> >> > Konstantin Knauf * konstantin.kn...@tngtech.com > >> <mailto:konstantin.kn...@tngtech.com> > >> >> <mailto:konstantin.kn...@tngtech.com > >> <mailto:konstantin.kn...@tngtech.com>> * +49-174-3413182 > >> <tel:%2B49-174-3413182> > >> >> <tel:%2B49-174-3413182> > >> >> > TNG Technology Consulting GmbH, Betastr. 13a, 85774 > >> Unterföhring > >> >> > Geschäftsführer: Henrik Klagges, Christoph Stock, Dr. > >> Robert Dahlke > >> >> > Sitz: Unterföhring * Amtsgericht München * HRB 135082 > >> >> > >> >> > >> > > >> > -- > >> > Konstantin Knauf * konstantin.kn...@tngtech.com > >> <mailto:konstantin.kn...@tngtech.com> * +49-174-3413182 > >> <tel:%2B49-174-3413182> > >> > TNG Technology Consulting GmbH, Betastr. 13a, 85774 Unterföhring > >> > Geschäftsführer: Henrik Klagges, Christoph Stock, Dr. Robert > Dahlke > >> > Sitz: Unterföhring * Amtsgericht München * HRB 135082 > >> > >> > > > > -- > Konstantin Knauf * konstantin.kn...@tngtech.com * +49-174-3413182 > TNG Technology Consulting GmbH, Betastr. 13a, 85774 Unterföhring > Geschäftsführer: Henrik Klagges, Christoph Stock, Dr. Robert Dahlke > Sitz: Unterföhring * Amtsgericht München * HRB 135082 >