Re: flink on yarn job always restart

2022-07-18 Thread SmileSmile
ne an OOM for a component like JM that doesn't run business logic (job parallelism is 3000, with multiple agg operations and sinks) Replied Message | From | Geng Biao | | Date | 07/18/2022 23:31 | | To | SmileSmile | | Cc | user | | Subject | Re: flink on yarn job always rest

Re: flink on yarn job always restart

2022-07-18 Thread Geng Biao
in this doc<https://help.aliyun.com/document_detail/411149.html#section-cco-ygc-hfe> ) due to wrong configuration but it may not be your case here. Best, Biao Geng From: SmileSmile Date: Monday, July 18, 2022 at 11:08 PM To: biaogeng7 Cc: user Subject: Re: flink on yarn job always r

Re: flink on yarn job always restart

2022-07-18 Thread SmileSmile
es it receive SIGNAL 15 2. is it because of some configuration? (e.g. deploy timeout causing kill?) Replied Message | From | Geng Biao | | Date | 07/18/2022 22:36 | | To | SmileSmile、user | | Cc | | | Subject | Re: flink on yarn job always restart | Hi, One possible direction is to check

Re: flink on yarn job always restart

2022-07-18 Thread Geng Biao
Hi, One possible direction is to check your YARN log or TM log to see if the YARN RM kills the TM for some reason(e.g. physical memory is over limit) and as a result, the JM will try to recover the TM repeatedly according to your restart strategy. The snippet of JM logs you provide is usually n

Re: flink on yarn job always restart

2022-07-18 Thread SmileSmile
. Replied Message | From | Zhanghao Chen | | Date | 07/18/2022 21:19 | | To | SmileSmile、user | | Cc | | | Subject | Re: flink on yarn job always restart | Hi, could you provide the whole JM log? Best, Zhanghao Chen From: SmileSmile Sent: Monday, July 18, 2022 20:46 To: user

Re: flink on yarn job always restart

2022-07-18 Thread Zhanghao Chen
Hi, could you provide the whole JM log? Best, Zhanghao Chen From: SmileSmile Sent: Monday, July 18, 2022 20:46 To: user Subject: flink on yarn job always restart hi all we meet a situation, parallelism 3000,the job contains multiple agg operation,the job recove

Re: Flink on YARN: delegation token expired prevent job restart

2020-11-18 Thread Truong Duc Kien
Hi all, So I've checked the log and it seems that the expired delegation error was triggered during resource localization. Maybe there's something wrong with my Hadoop setup, NMs are supposed to get a good token from RM in order to localize resources automatically. Regards, Kiên 2020-11-17 10:28

Re: Flink on YARN: delegation token expired prevent job restart

2020-11-17 Thread Kien Truong
Hi Yangze, Thanks for checking. I'm not using the new application mode, but the old single job yarn-cluster mode. I'll try to get some more logs tomorrow. Regards, Kien On 17 Nov 2020 at 16:37, Yangze Guo wrote: Hi, There is a login operation in YarnEntrypointUtils.logYarnEnvironmentInforma

Re: Flink on YARN: delegation token expired prevent job restart

2020-11-17 Thread Yangze Guo
Hi, There is a login operation in YarnEntrypointUtils.logYarnEnvironmentInformation without the keytab. One suspect is that Flink may access the HDFS when it tries to build the PackagedProgram. Does this issue only happen in the application mode? If so, I would cc @kkloudas. Best, Yangze Guo On

Re: Flink on YARN: delegation token expired prevent job restart

2020-11-17 Thread Yangze Guo
Hi, AFAIK, Flink does exclude the HDFS_DELEGATION_TOKEN in the HadoopModule when user provides the keytab and principal. I'll try to do a deeper investigation to figure out is there any HDFS access before the HadoopModule installed. Best, Yangze Guo On Tue, Nov 17, 2020 at 4:36 PM Kien Truong

Re: Flink on YARN: delegation token expired prevent job restart

2020-11-17 Thread Kien Truong
Hi, Yes, I did. There're also logs about logging in using keytab successfully in both Job Manager and Task Manager. I found some YARN docs about token renewal on AM restart > Therefore, to survive AM restart after token expiry, your AM has to get the NMs to localize the keytab or make no HDFS a

Re: Flink on YARN: delegation token expired prevent job restart

2020-11-17 Thread Yangze Guo
Hi, Kien, Do you config the "security.kerberos.login.principal" and the "security.kerberos.login.keytab" together? If you only set the keytab, it will not take effect. Best, Yangze Guo On Tue, Nov 17, 2020 at 3:03 PM Kien Truong wrote: > > Hi all, > > We are having an issue where Flink Applicat

Re: Flink on yarn : yarn-session understanding

2020-06-16 Thread Vikash Dat
yarn will assign a random port when flink is deployed. To get the port you need to do a yarn application -list and see the tracking url assigned to your flink cluster. The port in that url will be the port you need to use for the rest api. On Tue, Jun 16, 2020 at 08:49 aj wrote: > Ok, thanks for

Re: Flink on yarn : yarn-session understanding

2020-06-16 Thread aj
Ok, thanks for the clarification on yarn session. I am trying to connect to job manager on 8081 but it's not connecting. [image: image.png] So this is the address shown on my Flink job UI and i am trying to connect rest address on 8081 but its refusing connection. On Tue, Jun 9, 2020 at 1:03

Re: Flink on yarn : yarn-session understanding

2020-06-09 Thread Andrey Zagrebin
Hi Anuj, Afaik, the REST API should work for both modes. What is the issue? Maybe, some network problem to connect to YARN application master? Best, Andrey On Mon, Jun 8, 2020 at 4:39 PM aj wrote: > I am running some stream jobs that are long-running always. I am currently > submitting each jo

Re: Flink on yarn : yarn-session understanding

2020-06-08 Thread Xintong Song
Hi Anuj, By "standalone job on yarn", I assume you mean running one job per Flink cluster on Yarn, which is also known as job mode, or per-job mode? I'm asking because Flink has another standalone deployment mode [1], aside from the Yarn deployment mode. 1. The major difference between Flink Appl

Re: Flink On Yarn , ResourceManager is HA , if active ResourceManager changed,what is flink task status ?

2020-04-15 Thread Xintong Song
Normally, Yarn RM switch should not cause any problem to the running Flink instance. Unless the RM switch takes too long and Flink happens to request new containers during that time, it might lead to resource allocation timeout. Thank you~ Xintong Song On Wed, Apr 15, 2020 at 3:49 PM LakeShen

Re: Flink on YARN: Where to install Flink binaries?

2019-12-04 Thread Piper Piper
Thank you, Till! On Wed, Dec 4, 2019, 5:51 AM Till Rohrmann wrote: > Hi Piper, > > Answer 1: You should pick the Scala version you are using in your user > program. If you don't use Scala at all, then pick 2.11. > Answer 2: Flink does not need to be installed on the Yarn nodes. The > client is t

Re: Flink on YARN: Where to install Flink binaries?

2019-12-04 Thread Till Rohrmann
Hi Piper, Answer 1: You should pick the Scala version you are using in your user program. If you don't use Scala at all, then pick 2.11. Answer 2: Flink does not need to be installed on the Yarn nodes. The client is the machine from which you start the Flink cluster. The client machine needs to ha

Re: Flink on YARN: frequent "container released on a *lost* node"

2019-11-21 Thread Yang Wang
Hi Amran, >> Container released on a *lost* node If you see such exceptions, it means that the corresponding Yarn NodeManager has lost. So all the containers running on this node will be released. The Flink YarnResourceManager receives the 'lost' message from Yarn ResourceManager and will allocat

Re: Flink on YARN: frequent "container released on a *lost* node"

2019-11-21 Thread vino yang
Hi Amran, Did you monitor or have a look at your memory metrics(e.g. full GC) of your TM. There is a similar thread that a user reported the same question due to full GC, the link is here[1]. Best, Vino [1]: http://mail-archives.apache.org/mod_mbox/flink-user/201709.mbox/%3cfa4068d3-2696-4f29-8

Re: Flink on Yarn resource arrangement

2019-11-13 Thread vino yang
Hi Alex, Which Flink version are you using? AFAIK, since Flink 1.8+, the config option: "-yn" for Flink on YARN job cluster mode does not take effect(always 1 and would be overridden). So, the config option "-ys" and "-p" will decide the number of TM. The first example: -p(20)/-ys(3) should be

Re: flink on yarn-cluster kerberos authentication for hbase

2019-11-08 Thread Rong Rong
gt; *发送时间:* Friday, November 8, 2019 4:14 PM > *收件人:* venn > *抄送:* user@flink.apache.org > *主题:* Re: flink on yarn-cluster kerberos authentication for hbase > > > > Hello, > > > > Does this have helps to you? > > > https://stackoverflow.com/questions/3459

Re: flink on yarn-cluster kerberos authentication for hbase

2019-11-08 Thread Jaqie Chan
Hello, Does this have helps to you? https://stackoverflow.com/questions/34596165/how-to-do-kerberos-authentication-on-a-flink-standalone-installation Regards 嘉琪 On Fri, Nov 8, 2019 at 4:00 PM venn wrote: > HI Guys: > > Who can share some example for flink on yarn-cluster kerberos

Re: Flink on yarn use jar on hdfs

2019-09-18 Thread Yang Wang
Hi shengnan, Sorry for late. I will attach a pr to FLINK-13938 in this week. If we specify the shared lib(-ysl), all the jars located in the lib directory of flink client will not be uploaded. Instead, we will use the hdfs path to set the LocalResource of yarn. And the visibility of LocalResource

Re: Flink on yarn use jar on hdfs

2019-09-16 Thread Shengnan YU
And could you please share your github account with me? I am interested to follow you to see how you achieve this feature? Thank you. On 9/16/2019 14:44,Yang Wang wrote: Hi Shengnan, I think you mean to avoid uploading flink-dist jars in submission every time. I ha

Re: Flink on yarn use jar on hdfs

2019-09-15 Thread Yang Wang
Hi Shengnan, I think you mean to avoid uploading flink-dist jars in submission every time. I have created a JIRA[1] to use Yarn public cache to speed up the launch duration of JM and TM. After this feature merged, you could submit a flink job like below. ./bin/flink run -d -m yarn-cluster -p 20 -

Re: Flink on YARN: TaskManager heap auto-sizing?

2019-05-08 Thread Dylan Adams
Till, Thanks for the pointer to the code. Regards, Dylan On Wed, May 8, 2019 at 11:18 AM Till Rohrmann wrote: > Hi Dylan, > > the container's memory will be calculated here [1]. In the case of Yarn, > the user specifies the container memory size and based on this Flink > calculates with how mu

Re: Flink on YARN: TaskManager heap auto-sizing?

2019-05-08 Thread Till Rohrmann
Hi Dylan, the container's memory will be calculated here [1]. In the case of Yarn, the user specifies the container memory size and based on this Flink calculates with how much heap memory the JVM is started (container memory size - off heap memory - cut off memory). [1] https://github.com/apache

Re: flink on yarn log rolling

2019-03-20 Thread Paul Lam
Hi Shengnan, If you have the ssh permission, you could take a look at the container working directories to check if the taskmanager local log4j.properties is as expected. Best, Paul Lam > 在 2019年3月20日,15:30,Shengnan YU 写道: > > Hi all: > I'd like to enable log rolling for flink on yarn. I tri

Re: flink on yarn log rolling

2019-03-20 Thread Guowei Ma
Could you share the content of log4j.properties file you modified? And which version of Flink are you using? Best Guowei Shengnan YU 于2019年3月20日周三 下午3:30写道: > Hi all: > I'd like to enable log rolling for flink on yarn. I tried to modify > log4j.properties and logback.xml in flink/conf however t

Re: Flink on Yarn, restart job will not destroy original task manager

2018-09-03 Thread Gary Yao
In 1.5.0, I’ve not enable local > recovery. > > > > So whether I need manual disable local recovery via flink.conf? > > > > Regards > > > > James > > > > *From: *"James (Jian Wu) [FDS Data Platform]" > *Date: *Monday, September 3, 20

Re: Flink on Yarn, restart job will not destroy original task manager

2018-09-03 Thread James (Jian Wu) [FDS Data Platform]
eptember 3, 2018 at 4:13 PM To: Gary Yao Cc: "user@flink.apache.org" Subject: Re: Flink on Yarn, restart job will not destroy original task manager My Flink version is 1.5, I will rebuild new version flink Regards James From: Gary Yao Date: Monday, September 3, 2018 at 3:57 PM To: &

Re: Flink on Yarn, restart job will not destroy original task manager

2018-09-03 Thread James (Jian Wu) [FDS Data Platform]
My Flink version is 1.5, I will rebuild new version flink Regards James From: Gary Yao Date: Monday, September 3, 2018 at 3:57 PM To: "James (Jian Wu) [FDS Data Platform]" Cc: "user@flink.apache.org" Subject: Re: Flink on Yarn, restart job will not destroy original ta

Re: Flink on Yarn, restart job will not destroy original task manager

2018-09-03 Thread Gary Yao
Hi James, What version of Flink are you running? In 1.5.0, tasks can spread out due to changes that were introduced to support "local recovery". There is a mitigation in 1.5.1 that prevents task spread out but local recovery must be disabled [2]. Best, Gary [1] https://issues.apache.org/jira/bro

Re: [EXTERNAL] Re: Flink on YARN || Monitoring REST API Not Working || Please help

2018-01-31 Thread Raja . Aravapalli
highly helpful. Thanks. Regards, Raja. From: Jins George Date: Wednesday, January 31, 2018 at 8:51 PM To: Raja Aravapalli Cc: "user@flink.apache.org" Subject: [EXTERNAL] Re: Flink on YARN || Monitoring REST API Not Working || Please help 8081 is the default port for standalone cluster.

Re: Flink on YARN || Monitoring REST API Not Working || Please help

2018-01-31 Thread Jins George
8081 is the default port for standalone cluster. For Yarn flink cluster, Go to the Running applications and from the list of applications. You can get the Flink UI by clicking Application master link for the yarn session. Regards, Jins On Feb 1, 2018, at 8:06 AM, Raja.Aravapalli mailto:raja.a

Re: Flink on YARN

2018-01-21 Thread Mingmin Xu
In the Flink client node, you also need to have YARN configuration files which tells where's YARN. > Flink doc: the Client requires the YARN_CONF_DIR or HADOOP_CONF_DIR environment variable to be set to read the YARN and HDFS configuration. On Sat, Jan 20, 2018 at 3:54 AM, Kien Truong wrote: >

Re: Flink on YARN

2018-01-20 Thread Kien Truong
Hi, You only need to install Flink on the node where you want to perform job submission. Regards, Kien On 1/20/2018 3:23 PM, Soheil Pourbafrani wrote: Hi, I have a YARN cluster(containing no Flink installation) that I want to run Flink application on that. I was wondering if it is needed

Re: Flink on YARN - tmp directory

2017-07-31 Thread Aljoscha Krettek
Hi Chris, I think in this case we need to change what is passed as "-Djava.io .tmpdir" to the JVMs that run the TaskManagers. You should be able to achieve this via env.java.opts or more specifically env.java.opts.taskmanager [1]. The directory specified via task taskmanager.

Re: Flink on YARN - tmp directory

2017-07-28 Thread Chris Hebert
I should also note that the above steps did get the Flink JobManager and TaskManagers to save their tmp web dashboard files to /my/tmp/ and to show in the Dashboard that the taskmanager.tmp.dirs property had been properly set to /my/tmp/, but the tmp files I wrote in my jobs stubbornly wrote to /tm

Re: flink on yarn ha

2017-02-23 Thread Robert Metzger
Hi, This looks like a shading issue. Can you post the classpath the JobManager / AppMaster is logging on startup on the mailing list? If seems that Hadoop loads an unshaded version of the SecurityProtos. Maybe there is some hadoop version mixup. Are you using a Hadoop distribution (like CDH or HD

Re: flink on yarn ha

2017-02-21 Thread lining jing
Hi, I update flink from 1.1.3 to 1.2 but fail this is jobManager error log Failed toString() invocation on an object of type [org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl] java.lang.NoSuchMethodError: org.apache.hadoop.security.proto.SecurityProtos.getDescriptor()Lorg/apache/fl

Re: flink on yarn ha

2017-02-21 Thread lining jing
Thanks, Stephan ! I will try it! 2017-02-21 21:42 GMT+08:00 Stephan Ewen : > Hi! > > Flink 1.1.4 and Flink 1.2 fixed a bunch of issues with HA, can you try > those versions? > > If these also have issues, could you share the logs of the JobManager? > > Thanks! > > On Tue, Feb 21, 2017 at 11:41 A

Re: flink on yarn ha

2017-02-21 Thread Stephan Ewen
Hi! Flink 1.1.4 and Flink 1.2 fixed a bunch of issues with HA, can you try those versions? If these also have issues, could you share the logs of the JobManager? Thanks! On Tue, Feb 21, 2017 at 11:41 AM, lining jing wrote: > flink version: 1.1.3 > > kill jobmanager, the job fail. Ha config di

Re: Flink on YARN: Cannot connect to JobManager

2017-01-15 Thread Till Rohrmann
Hi Malte, can it be that you’re trying to request more resources from your yarn cluster than there are currently available? It depends a little bit on your other settings but -yn 2 says that you request 2 TaskManagers. Additionally, Flink will also allocate another container for the JobManager. Pe

Re: Flink on Yarn delegation token renewal

2016-11-08 Thread Theofilos Kakantousis
Thank you for the prompt reply Stefan! Cheers, Theo On 2016-11-08 11:29, Stefan Richter wrote: Hi, I think this problem tracked in this issue: https://issues.apache.org/jira/browse/FLINK-3670 . This means that the current master and upcoming release 1.2 should work correctly. Best, Stefan

Re: Flink on Yarn delegation token renewal

2016-11-08 Thread Stefan Richter
Hi, I think this problem tracked in this issue: https://issues.apache.org/jira/browse/FLINK-3670 . This means that the current master and upcoming release 1.2 should work correctly. Best, Stefan > Am 08.11.2016 um 10:25 schrieb Theofilos Kaka

Re: Flink on YARN - Fault Tolerance | use case supported or not

2016-11-01 Thread Anchit Jatana
Yes, thank Stephan. Regards, Anchit -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Flink-on-YARN-Fault-Tolerance-use-case-supported-or-not-tp9776p9817.html Sent from the Apache Flink User Mailing List archive. mailing list archive at Nabbl

Re: Flink on YARN - Fault Tolerance | use case supported or not

2016-10-31 Thread Stephan Ewen
Hi Anchit! In high-availability cases, a Flink cluster recovers jobs that it considers belonging to the cluster. That is determined by what is set in the Zookeeper Cluster Namespace: "recovery.zookeeper.path.namespace" https://github.com/apache/flink/blob/release-1.1.3/flink-core/src/main/java/org

Re: Flink on YARN - Fault Tolerance | use case supported or not

2016-10-31 Thread Kostas Kloudas
Hi Jatana, As you pointed out, the correct way to do the above is to use savepoints. If you kill your application, then this is not a crass but rather a voluntary action. I am also looping in Max, as he may have something more to say on this. Cheers, Kostas On Sat, Oct 29, 2016 at 12:13 AM, Anc

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-23 Thread Maximilian Michels
Hi Mira, Does using the fully-qualified hostname solve the issue? Thanks, Max On Mon, Aug 22, 2016 at 1:38 PM, Miroslav Gajdoš wrote: > Here is the log from yarn application - run on another cluster (this > time cdh5.7.0, but with similar configuration). Check the hostnames; in > configuration,

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-22 Thread Miroslav Gajdoš
Here is the log from yarn application - run on another cluster (this time cdh5.7.0, but with similar configuration). Check the hostnames; in configuration, there are aliases used and the difference from fqdn may be the cause, judging by the log (exception at line 87)... http://pastebin.com/iimPVbX

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-19 Thread Maximilian Michels
Hi Mira, If I understood correctly, the log output should be for Flink 1.1.1. However, there are classes present in the log which don't exist in Flink 1.1.1, e.g. FlinkYarnClient. Could you please check if you posted the correct log? Also, it would be good to have not only the client log but also

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-18 Thread Miroslav Gajdoš
Tried to build it from source as well as use prebuilt binary release (v1.1.1), the last one produced this log output: http://pastebin.com/3L5Yhs9x Application in yarn still fails on "Fatal error in AM: The ContainerLaunchContext was not set". Mira Miroslav Gajdoš píše v Čt 18. 08. 2016 v 10:36 +

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-18 Thread Miroslav Gajdoš
Hi Max, we are building it from sources and package it for debian. I can try to use the binary release for hadoop 2.6.0. Regarding zookeeper, we do not share instances between dev and production. Thanks, Miroslav Maximilian Michels píše v Čt 18. 08. 2016 v 10:17 +0200: > Hi Miroslav, > > From

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-18 Thread Maximilian Michels
Hi Miroslav, >From the logs it looks like you're using Flink version 1.0.x. The ContainerLaunchContext is always set by Flink. I'm wondering why this error can still occur. Are you using the default Hadoop version that comes with Flink (2.3.0)? You could try the Hadoop 2.6.0 build of Flink. Does

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-16 Thread Miroslav Gajdoš
Log from yarn session runner is here: http://pastebin.com/xW1W4HNP Our hadoop distribution is from cloudera, resourcenanager version: 2.6.0-cdh5.4.5, it runs in HA mode (there could be some redirecting on accessing resourcemanager and/or namenode to active one). Ufuk Celebi píše v Út 16. 08. 2016

Re: flink on yarn - Fatal error in AM: The ContainerLaunchContext was not set

2016-08-16 Thread Ufuk Celebi
This could be a bug in Flink. Can you share the complete logs of the run? CC'ing Max who worked on the YARN client recently who might have an idea in which cases Flink would not set the context. On Tue, Aug 16, 2016 at 11:00 AM, Miroslav Gajdoš wrote: > Hi guys, > > i've run into some problems wi

Re: Flink on YARN - how to resize a running cluster?

2016-06-30 Thread Till Rohrmann
Yes that's the way to go at the moment. Cheers, Till On Thu, Jun 30, 2016 at 12:47 PM, Márton Balassi wrote: > Hi Josh, > > Yes, currently that is a reasonable workaround. > > Best, > > Marton > > On Thu, Jun 30, 2016 at 12:38 PM, Josh wrote: > >> Hi Till, >> >> Thanks, that's very helpful! >>

Re: Flink on YARN - how to resize a running cluster?

2016-06-30 Thread Márton Balassi
Hi Josh, Yes, currently that is a reasonable workaround. Best, Marton On Thu, Jun 30, 2016 at 12:38 PM, Josh wrote: > Hi Till, > > Thanks, that's very helpful! > So I guess in that case, since it isn't possible to increase the job > parallelism later, it might be sensible to use say 10x the p

Re: Flink on YARN - how to resize a running cluster?

2016-06-30 Thread Josh
Hi Till, Thanks, that's very helpful! So I guess in that case, since it isn't possible to increase the job parallelism later, it might be sensible to use say 10x the parallelism that I need right now (even if only running on a couple of task managers) - so that it's possible to scale the job in th

Re: Flink on YARN - how to resize a running cluster?

2016-06-30 Thread Till Rohrmann
Hi Josh, at the moment it is not possible to dynamically increase the parallelism of your job. The same holds true for a restarting a job from a savepoint. But we're currently working on exactly this. So in order to change the parallelism of your job, you would have to restart the job from scratch

Re: Flink on YARN : Amazon S3 wrongly used instead of HDFS

2016-06-20 Thread Robert Metzger
Hi Charles, sorry for the late response. I put an answer on Stack Overflow. Regards, Robert On Fri, Jun 17, 2016 at 3:11 PM, VALLEE Charles wrote: > Hi everyone, > > I followed Flink on YARN's setup documentation >

Re: Flink on Yarn - ApplicationMaster command

2016-04-25 Thread Maximilian Michels
Great to hear! :) On Sun, Apr 24, 2016 at 3:51 PM, Theofilos Kakantousis wrote: > Hi, > > The issue was a mismatch of jar versions on my client. Seems to be working > fine now. > Thanks again for your help! > > Cheers, > Theofilos > > > On 2016-04-22 18:22, Theofilos Kakantousis wrote: > > Hi Max

Re: Flink on Yarn - ApplicationMaster command

2016-04-24 Thread Theofilos Kakantousis
Hi, The issue was a mismatch of jar versions on my client. Seems to be working fine now. Thanks again for your help! Cheers, Theofilos On 2016-04-22 18:22, Theofilos Kakantousis wrote: Hi Max, I manage to get the jobManagerAddress from FlinkYarnCluster, however when I submit a job using t

Re: Flink on Yarn - ApplicationMaster command

2016-04-22 Thread Theofilos Kakantousis
Hi Max, I manage to get the jobManagerAddress from FlinkYarnCluster, however when I submit a job using the code below the jobID is null. Is there something wrong in the way I submit the job? Otherwise any ideas to which direction should I further investigate? The /runBlocking /call returns al

Re: Flink on Yarn - ApplicationMaster command

2016-04-22 Thread Maximilian Michels
Hi Theofilos, Assuming you have the FlinkYarnCluster after the call to deploy(). You can get the JobManager address using the InetSocketAddress address = cluster.getJobManagerAddress(); Then create a Configuration with this address: Configuration config = new Configuration(); config.setString(C

Re: Flink on Yarn - ApplicationMaster command

2016-04-19 Thread Theofilos Kakantousis
Hi Max, Thank you for your reply. Exactly, I want to setup the Yarn cluster and submit a job through code and not using cmd client. I had done what you suggested, I used part of the deploy method to write my own code that starts up the cluster which seems to be working fine. Could you point m

Re: Flink on Yarn - ApplicationMaster command

2016-04-19 Thread Maximilian Michels
Hi Theofilos, I'm not sure whether I understand correctly what you are trying to do. I'm assuming you don't want to use the command-line client. You can setup the Yarn cluster in your code manually using the FlinkYarnClient class. The deploy() method will give you a FlinkYarnCluster which you can

Re: Flink on YARN: long-running session vs. one-off jobs

2016-03-07 Thread Maximilian Michels
For the per-job cluster: Yes, the JobManager is started exclusively for the job. For the Yarn session: No, the JobManager stays alive during the entire session and may execute one or more jobs (one after another or even at the same time). On Mon, Mar 7, 2016 at 6:37 PM, Stefano Baghino wrote: >

Re: Flink on YARN: long-running session vs. one-off jobs

2016-03-07 Thread Stefano Baghino
One last question: running multiple jobs mean that each one has its own JobManager, right? On Mon, Mar 7, 2016 at 3:14 PM, Stefano Baghino < stefano.bagh...@radicalbit.io> wrote: > Good, thank you for the explanation! > > On Mon, Mar 7, 2016 at 2:38 PM, Maximilian Michels wrote: > >> Hi Stefano,

Re: Flink on YARN: long-running session vs. one-off jobs

2016-03-07 Thread Stefano Baghino
Good, thank you for the explanation! On Mon, Mar 7, 2016 at 2:38 PM, Maximilian Michels wrote: > Hi Stefano, > > Essentially the Yarn Session is not much different from a per-job Yarn > cluster. In either case, a Flink cluster is brought up with resources > provided by Yarn. In case of the Yarn

Re: Flink on YARN: long-running session vs. one-off jobs

2016-03-07 Thread Maximilian Michels
Hi Stefano, Essentially the Yarn Session is not much different from a per-job Yarn cluster. In either case, a Flink cluster is brought up with resources provided by Yarn. In case of the Yarn session this cluster doesn't do anything until a job is submitted. In case of the per-job Yarn cluster, a j

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-08 Thread Pieter Hameete
Solved: indeed it needed to be built for YARN 2.7.1 specifically. Cheers! 2016-02-08 19:13 GMT+01:00 Robert Metzger : > Mh, that's weird. Maybe both resource managers are marked as "standby"? > Not sure what can cause this issue. > > Which YARN version are you using? Maybe you need to build Flink

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-08 Thread Robert Metzger
Mh, that's weird. Maybe both resource managers are marked as "standby"? Not sure what can cause this issue. Which YARN version are you using? Maybe you need to build Flink against that specific hadoop version yourself. On Mon, Feb 8, 2016 at 5:50 PM, Pieter Hameete wrote: > After downloading an

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-08 Thread Pieter Hameete
After downloading and building the 1.0-SNAPSHOT from the master branch I do run into another problem when starting a YARN cluster. The startup now infinitely loops at the following step: 17:39:12,369 INFO org.apache.hadoop.yarn.client.ConfiguredRMFailoverProxyProvider - Failing over to rm2 17:39:

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-08 Thread Pieter Hameete
Ive tried setting the yarn.application-master.port property in flink-conf.yaml to a range suggested in https://ci.apache.org/projects/flink/flink-docs-master/setup/yarn_setup.html#running-flink-on-yarn-behind-fi rewalls The JobManager does not seem to be picking the property up. Am I setting this

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-08 Thread Pieter Hameete
Matter of RTFM eh ;-) thx and sorry for the bother. 2016-02-08 17:06 GMT+01:00 Robert Metzger : > You said earlier that you are using Flink 0.10. The feature is only > available in 1.0-SNAPSHOT. > > On Mon, Feb 8, 2016 at 4:53 PM, Pieter Hameete wrote: > >> Ive tried setting the yarn.application

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-08 Thread Robert Metzger
You said earlier that you are using Flink 0.10. The feature is only available in 1.0-SNAPSHOT. On Mon, Feb 8, 2016 at 4:53 PM, Pieter Hameete wrote: > Ive tried setting the yarn.application-master.port property in > flink-conf.yaml to a range suggested in > https://ci.apache.org/projects/flink/f

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-07 Thread Pieter Hameete
I found the relevant information on the website. Ill consult with the cluster admin tomorrow, thanks for the help :-) - Pieter 2016-02-07 19:31 GMT+01:00 Robert Metzger : > Hi, > > we had other users with a similar issue as well. There is a configuration > value which allows you to specify a sin

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-07 Thread Robert Metzger
Hi, we had other users with a similar issue as well. There is a configuration value which allows you to specify a single port or a range of ports for the JobManager to allocate when running on YARN. Note that when using this with a single port, the JMs may collide. On Sun, Feb 7, 2016 at 7:25 P

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-07 Thread Pieter Hameete
Hi Stephan, surely it seems this way! I must not be the first with this issue though? I'll have to contact the cluster admins to find a solution together. What would be a way of make the JobManagers accessible from outside the network, because the IP and port number changes every time. Alternativ

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-06 Thread Stephan Ewen
Yeah, sounds a lot like the client cannot connect to the JobManager port. The ports to communicate with HDFS and the YARN resource manager may be whitelisted r forwarded, so you can submit the YARN session, but then not connect to the JobManager afterwards. On Sat, Feb 6, 2016 at 2:11 PM, Piete

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-06 Thread Pieter Hameete
Hi Max! I'm using Flink 0.10.1 and indeed the cluster seems to be created fine, all in the JobManager Web UI looks good. It seems like the JobManager initiates the connection with my VM and cannot reach it. It could be that this is similar to the problem here: http://apache-spark-user-list.10015

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-06 Thread Maximilian Michels
Hi Pieter, Which version of Flink are you using? It appears you've created a Flink YARN cluster but you can't reach the JobManager afterwards. Cheers, Max On Sat, Feb 6, 2016 at 1:42 PM, Pieter Hameete wrote: > Hi Robert, > > unfortunately there are no signs of what is going wrong in the logs.

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-06 Thread Pieter Hameete
Hi Robert, unfortunately there are no signs of what is going wrong in the logs. The last log messages are about succesful registration of the TaskManagers. I'm also fairly sure it must be something in my VM that is causing this, because when I start the yarn-session from a login node that is on t

Re: Flink on YARN: Stuck on "Trying to register at JobManager"

2016-02-06 Thread Robert Metzger
Hi, did you check the logs of the JobManager itself? Maybe it'll tell us already whats going on. On Sat, Feb 6, 2016 at 12:14 PM, Pieter Hameete wrote: > Hi Guys! > > Im attempting to run Flink on YARN, but I run into an issue. Im starting > the Flink YARN session from an Ubuntu 14.04 VM. All g

Re: flink on yarn configuration

2015-07-14 Thread Robert Metzger
Hi Paul, I don't think you need 10 GB of heap space for the JobManager. Usually 1 GB are sufficient. Since you have 3 nodes, I would start Flink with 3 task managers. I think you can also launch such a cluster: ./flink-0.9.0/bin/yarn-session.sh -n 3 -jm 1024 -tm 13000 Regarding the memory you are

Re: flink on yarn configuration

2015-07-14 Thread Till Rohrmann
Hi Paul, when you run your Flink cluster with YARN then we cannot give the full amount of the allocated container memory to Flink. The reason is that YARN itself needs some of the memory as well. Since YARN is quite strict with containers which exceed their memory limit (the container is instantly