Re: Understanding RocksDBStateBackend in Flink on Yarn on AWS EMR

2024-03-26 Thread Yang Wang
Usually, you should use the HDFS nameservice instead of the NameNode hostname:port to avoid NN failover. And you could find the supported nameservice in the hdfs-site.xml in the key *dfs.nameservices*. Best, Yang On Fri, Mar 22, 2024 at 8:33 PM Sachin Mittal wrote: > So, when we create an EMR

Re: Understanding RocksDBStateBackend in Flink on Yarn on AWS EMR

2024-03-22 Thread Sachin Mittal
So, when we create an EMR cluster the NN service runs on the primary node of the cluster. Now at the time of creating the cluster, how can we specify the name of this NN in format hdfs://*namenode-host*:8020/. Is there a standard name by which we can identify the NN server ? Thanks Sachin On Fr

Re: Understanding RocksDBStateBackend in Flink on Yarn on AWS EMR

2024-03-21 Thread Asimansu Bera
Hello Sachin, Typically, Cloud VMs are ephemeral, meaning that if the EMR cluster goes down or VMs are required to be shut down for security updates or due to faults, new VMs will be added to the cluster. As a result, any data stored in the local file system, such as file://tmp, would be lost. To

Understanding RocksDBStateBackend in Flink on Yarn on AWS EMR

2024-03-21 Thread Sachin Mittal
Hi, We are using AWS EMR where we can submit our flink jobs to a long running flink cluster on Yarn. We wanted to configure RocksDBStateBackend as our state backend to store our checkpoints. So we have configured following properties in our flink-conf.yaml - state.backend.type: rocksdb - s

Re: Uneven TM Distribution of Flink on YARN

2023-09-06 Thread Lu Niu
en Zhanghao >>> *发送时间:* Tuesday, August 29, 2023 12:14:53 PM >>> *收件人:* Lu Niu ; Weihua Hu >>> *抄送:* Kenan Kılıçtepe ; user < >>> user@flink.apache.org> >>> *主题:* Re: Uneven TM Distribution of Flink on YARN >>> >>> CCing @Weihua Hu

Re: Uneven TM Distribution of Flink on YARN

2023-09-06 Thread Lu Niu
*From: *Lu Niu > *Date: *Thursday, September 7, 2023 at 12:17 AM > *To: *Geng Biao > *Cc: *Chen Zhanghao , Weihua Hu < > huweihua@gmail.com>, Kenan Kılıçtepe , user < > user@flink.apache.org> > *Subject: *Re: Uneven TM Distribution of Flink on YARN > > Hi, Than

Re: Uneven TM Distribution of Flink on YARN

2023-09-06 Thread Biao Geng
duling strategy, the final distribution of apps after some time is different. Best, Biao Geng From: Lu Niu Date: Thursday, September 7, 2023 at 12:17 AM To: Geng Biao Cc: Chen Zhanghao , Weihua Hu , Kenan Kılıçtepe , user Subject: Re: Uneven TM Distribution of Flink on YARN Hi, Thanks for all you

Re: Uneven TM Distribution of Flink on YARN

2023-09-06 Thread Lu Niu
:53 PM >> *收件人:* Lu Niu ; Weihua Hu >> *抄送:* Kenan Kılıçtepe ; user > > >> *主题:* Re: Uneven TM Distribution of Flink on YARN >> >> CCing @Weihua Hu , who is an expert on this. Do >> you have any ideas on the phenomenon here? >> >> Best, &g

Re: Uneven TM Distribution of Flink on YARN

2023-08-30 Thread Lu Niu
iOS <https://aka.ms/o0ukef> > -- > *发件人:* Chen Zhanghao > *发送时间:* Tuesday, August 29, 2023 12:14:53 PM > *收件人:* Lu Niu ; Weihua Hu > *抄送:* Kenan Kılıçtepe ; user > *主题:* Re: Uneven TM Distribution of Flink on YARN > > CCing @Weihua Hu , who is an expert on this.

Re: Uneven TM Distribution of Flink on YARN

2023-08-29 Thread Geng Biao
er 主题: Re: Uneven TM Distribution of Flink on YARN CCing @Weihua Hu<mailto:huweihua@gmail.com> , who is an expert on this. Do you have any ideas on the phenomenon here? Best, Zhanghao Chen From: Lu Niu Sent: Tuesday, August 29, 2023 12:11:35 PM To: C

Re: Uneven TM Distribution of Flink on YARN

2023-08-28 Thread Chen Zhanghao
ct: Re: Uneven TM Distribution of Flink on YARN Thanks for your reply. The interesting fact is that we also managed spark on yarn. However. Only the flink cluster is having the issue. I am wondering whether there is a difference in the implementation on flink side. Best Lu On Mon, Aug 28, 2023 at 8

Re: Uneven TM Distribution of Flink on YARN

2023-08-28 Thread Lu Niu
ts Standalone mode Flink > clusters, and does not take effect on a Flink cluster on YARN. > > Best, > Zhanghao Chen > -- > *发件人:* Lu Niu > *发送时间:* 2023年8月29日 4:30 > *收件人:* Kenan Kılıçtepe > *抄送:* user > *主题:* Re: Uneven TM Distribution of Flink

回复: Uneven TM Distribution of Flink on YARN

2023-08-28 Thread Chen Zhanghao
mode Flink clusters, and does not take effect on a Flink cluster on YARN. Best, Zhanghao Chen 发件人: Lu Niu 发送时间: 2023年8月29日 4:30 收件人: Kenan Kılıçtepe 抄送: user 主题: Re: Uneven TM Distribution of Flink on YARN Thanks for the reply. We've already set cluster.e

Re: Uneven TM Distribution of Flink on YARN

2023-08-28 Thread Lu Niu
Thanks for the reply. We've already set cluster.evenly-spread-out-slots = true Best Lu On Mon, Aug 28, 2023 at 1:23 PM Kenan Kılıçtepe wrote: > Have you checked config param cluster.evenly-spread-out-slots ? > > > On Mon, Aug 28, 2023 at 10:31 PM Lu Niu wrote: > >> Hi, Flink users >> >> We hav

Re: Uneven TM Distribution of Flink on YARN

2023-08-28 Thread Kenan Kılıçtepe
Have you checked config param cluster.evenly-spread-out-slots ? On Mon, Aug 28, 2023 at 10:31 PM Lu Niu wrote: > Hi, Flink users > > We have recently observed that the allocation of Flink TaskManagers in our > YARN cluster is not evenly distributed. We would like to hear your thoughts > on this

Uneven TM Distribution of Flink on YARN

2023-08-28 Thread Lu Niu
Hi, Flink users We have recently observed that the allocation of Flink TaskManagers in our YARN cluster is not evenly distributed. We would like to hear your thoughts on this matter. 1. Our setup includes Flink version 1.15.1 and Hadoop 2.10.0. 2. The uneven distribution is that out of a 370-node

Re: Web UI don't show up In Flink on Yarn (Flink 1.17)

2023-05-25 Thread Weihua Hu
t; and this info came from yarn resoucemanager > > 获取 Outlook for iOS <https://aka.ms/o0ukef> > -- > *发件人:* tan yao > *发送时间:* Thursday, May 25, 2023 8:14:45 PM > *收件人:* Weihua Hu > *抄送:* user > *主题:* Re: Web UI don't show up In

Re: Web UI don't show up In Flink on Yarn (Flink 1.17)

2023-05-25 Thread tan yao
and this info came from yarn resoucemanager 获取 Outlook for iOS<https://aka.ms/o0ukef> 发件人: tan yao 发送时间: Thursday, May 25, 2023 8:14:45 PM 收件人: Weihua Hu 抄送: user 主题: Re: Web UI don't show up In Flink on Yarn (Flink 1.17) yes i have tried ip dire

Re: Web UI don't show up In Flink on Yarn (Flink 1.17)

2023-05-25 Thread Weihua Hu
Hi, Are there any reported exceptions? Did you try using curl to query the rest API, such as "curl http://{ip:port}/overview"; Best, Weihua On Thu, May 25, 2023 at 8:49 AM tan yao wrote: > Hi all, > I find a strange thing with flink 1.17 deployed on yarn (CDH 6.x), flink > web ui can not show

Web UI don't show up In Flink on Yarn (Flink 1.17)

2023-05-24 Thread tan yao
Hi all,   I find a strange thing with flink 1.17 deployed on yarn (CDH 6.x), flink web ui can not show up from yarn web link "ApplicationMaster",even typed jobmanager ip directly in browser . when i run wordcount application in flink 1.17 examples, and click yarn web "ApplicationMaster" link

Re: Deploy Flink on YARN or Kubernetes.

2022-12-20 Thread Biao Geng
some observations of myself and hope it can provide more information for the discussion: 1. stability: Flink on YARN module and Hadoop ecosystem have developed for a longer period of time than Flink on K8S and K8S ecosystem. The codebase of Flink on YARN module is more stable and it could be easier to get rel

Re: Deploy Flink on YARN or Kubernetes.

2022-12-20 Thread Márton Balassi
Hi Ruibin, Given that you are starting fresh I would recommend going with Kubernetes and specifically checking out the Flink Kubernetes Operator. [1] I have worked with Yarn for years before I transitioned to Kubernetes a year ago and I am pleased that we made the jump. To address you point on a v

Deploy Flink on YARN or Kubernetes.

2022-12-18 Thread Ruibin Xing
Hi all, We are currently setting up a new Flink cluster and are trying to decide on the best deployment method. As far as we know, Flink supports two resource providers: YARN and Kubernetes. We are having difficulty evaluating the pros and cons of each provider, particularly in terms of stability,

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
not the root cause. Best, Biao Geng From: SmileSmile Date: Monday, July 18, 2022 at 8:46 PM 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 recover from checkpoint or savepoint must be unrecoverable

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

flink on yarn job always restart

2022-07-18 Thread SmileSmile
hi all we meet a situation, parallelism 3000,the job contains multiple agg operation,the job recover from checkpoint or savepoint must be unrecoverable, the job restarts repeatedly jm error logorg.apache.flink.runtime.entrypoint.ClusterEntrypoint[] - RECEIVED S IGNAL 15: SIGTERM. Shuttin

RE: Upgrading from Flink on YARN 1.9 to 1.11

2021-08-20 Thread Hailu, Andreas [Engineering]
things up. // ah From: David Morávek Sent: Tuesday, August 17, 2021 4:37 AM To: Hailu, Andreas [Engineering] Cc: Ravichandran, Soorya Prasanna [Engineering] ; user@flink.apache.org Subject: Re: Upgrading from Flink on YARN 1.9 to 1.11 Hi Andreas, the problem here is that the command you're

Re: Upgrading from Flink on YARN 1.9 to 1.11

2021-08-17 Thread David Morávek
11.2_POC. The logs are quite brief – would you be able to > have a look at see if you can see if there’s something we’re doing that’s > clearly wrong? > > > > Something I did notice is that with the upgrade, our submissions are now > using the introduction of this ContextEnvir

RE: Upgrading from Flink on YARN 1.9 to 1.11

2021-08-16 Thread Hailu, Andreas [Engineering]
David Morávek Sent: Monday, August 16, 2021 6:28 AM To: Hailu, Andreas [Engineering] Cc: user@flink.apache.org Subject: Re: Upgrading from Flink on YARN 1.9 to 1.11 Hi Andreas, Per-job and session deployment modes should not be affected by this FLIP. Application mode is just a new deployment mo

Re: Upgrading from Flink on YARN 1.9 to 1.11

2021-08-16 Thread David Morávek
Hi Andreas, Per-job and session deployment modes should not be affected by this FLIP. Application mode is just a new deployment mode (where job driver runs embedded within JM), that co-exists with these two. >From information you've provided, I'd say your actual problem is this exception: ``` C

Upgrading from Flink on YARN 1.9 to 1.11

2021-08-13 Thread Hailu, Andreas [Engineering]
Hello folks! We're looking to upgrade from 1.9 to 1.11. Our Flink applications run on YARN and each have their own clusters, with each application having multiple jobs submitted. Our current submission command looks like this: $ run -m yarn-cluster --class com.class.name.Here -p 2 -yqu queue-na

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

Flink on YARN: delegation token expired prevent job restart

2020-11-16 Thread Kien Truong
Hi all, We are having an issue where Flink Application Master is unable to automatically restart Flink job after its delegation token has expired. We are using Flink 1.11 with YARN 3.1.1 in single job per yarn-cluster mode. We have also add valid keytab configuration and taskmanagers are able to

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

Flink on yarn : yarn-session understanding

2020-06-08 Thread aj
I am running some stream jobs that are long-running always. I am currently submitting each job as a standalone job on yarn. 1. I need to understand what is the advantage of using yarn-session and when should I use that. 2. Also, I am not able to access rest API services is it because I am running

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

2020-04-15 Thread Xintong Song
wrote: > Hi community, > > I have a question about flink on yarn ha , if active resourcemanager > changed, what is the flink task staus. Is flink task running normally? > Should I must restart my flink task to run? > > Thanks to your reply. > > Best, > LakeShen >

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

2020-04-15 Thread LakeShen
Hi community, I have a question about flink on yarn ha , if active resourcemanager changed, what is the flink task staus. Is flink task running normally? Should I must restart my flink task to run? Thanks to your reply. Best, LakeShen

Re: REST rescale with Flink on YARN

2020-01-28 Thread Yang Wang
Gary is right. You could also access the Yarn RM rest api to get the AM original address. http://hadoop.apache.org/docs/r2.8.5/hadoop-yarn/hadoop-yarn-site/ResourceManagerRest.html#Cluster_Application_API Best, Yang Gary Yao 于2020年1月28日周二 下午6:17写道: > Hi, > > You can use > > yarn applicati

Re: REST rescale with Flink on YARN

2020-01-28 Thread Gary Yao
Hi, You can use yarn application -status to find the host and port that the server is listening on (AM host & RPC Port). If you need to access that information programmatically, take a look at the YarnClient [1]. Best, Gary [1] https://hadoop.apache.org/docs/r2.8.5/api/org/apache/hadoop/

Re: REST rescale with Flink on YARN

2020-01-23 Thread Vasily Melnik
Hi all, I've found some solution for this issue. Problem is that with YARN ApplicationMaster URL we communicate with JobManager via proxy which is implemented on Jetty 6 (for Hadoop 2.6). So to use PATCH method we need to locate original JobManager URL. Using /jobmanager/config API we could get onl

Re: REST rescale with Flink on YARN

2020-01-23 Thread Chesnay Schepler
Older versions of Jetty don't support PATCH requests. You will either have to update it or create a custom Flink version that uses POST for the rescale operation. On 23/01/2020 13:23, Vasily Melnik wrote: Hi all. I'm using Flink 1.8 on YARN with CDH 5.12 When i try to perform rescale request:

REST rescale with Flink on YARN

2020-01-23 Thread Vasily Melnik
Hi all. I'm using Flink 1.8 on YARN with CDH 5.12 When i try to perform rescale request: curl -v -X PATCH '/proxy/application_1576854986116_0079/jobs/11dcfc3163936fc019e049fc841b075b/rescaling?parallelism=3

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

Flink on YARN: Where to install Flink binaries?

2019-12-04 Thread Piper Piper
Hello, I have a YARN/Hadoop 2.7.6 cluster, on which I plan to run Flink in Job mode using: Flink 1.9.1 (with Flink application programs written in Java) Prebundled Hadoop 2.7.5 Question 1: Which scala version must I choose for the Flink 1.9.1 binary (2.11 or 2.12)? Secondly, I had read a documen

flink on yarn 的 kerberos 认证问题

2019-11-25 Thread venn
各位大佬: 请教一个flink 认证的问题: Flink on yarn 运行在不用认证的 Hadoop 集群上,怎样访问带kerberos 认证集群的 hbase ? 下面是一些我们使用的描述和发现的问题: 我们有两个hadoop 集群,一个使用 Kerberos 认证模式,一个是 simple 认证模式,Flink 1.9.0 部署在 simple 认证的集群上。 最近在使用flink 读取 Kerberos 认证的集群的 hbase 上遇到了问题。配置 flink-conf.yaml 中的配置参数

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

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

2019-11-21 Thread amran dean
Hello, I am frequently seeing this error in my jobmanager logs: 2019-11-18 09:07:08,863 INFO org.apache.flink.runtime.executiongraph.ExecutionGraph- Source: kdi_g2 -> (Sink: s_id_metadata, Sink: t_id_metadata) (23/24) (5e10e88814fe4ab0be5f554ec59bd93d) switched from RUNNING to FAILED. jav

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 ex

Flink on Yarn resource arrangement

2019-11-13 Thread qq
Hi all, Could you list details how Flink job on Yarn resources managed ? I used command “-p 20 -yn 5 -ys 3 -yjm 2048m -ytm 2048m” to run flink job. I got containers vcores 8 22 Task Managers 7 Total Task Slots 21 I used command “-p 20 -yn 7 -ys 4 -yjm 2048m -ytm 2048m” to

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

回复: flink on yarn-cluster kerberos authentication for hbase

2019-11-08 Thread venn
Thanks, I already seen, not work for me 发件人: Jaqie Chan 发送时间: 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/34596165/how

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

flink on yarn-cluster kerberos authentication for hbase

2019-11-08 Thread venn
HI Guys: Who can share some example for flink on yarn-cluster kerberos authentication for hbase I auth as what I do in java program, it look like just authentication zookeeper, cannot access for hbase thanks

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 -

Flink on yarn use jar on hdfs

2019-09-15 Thread Shengnan YU
Hi everyone! I found that everytime I start a flink-yarn application, client will ship flink-uber jar and other dependencies to hdfs and start appMaster. Is there any approaches to locate flink-uber jar and other library jars on hdfs and let only configuration file being shipped. Therefore the y

Re: How can TMs distribute evenly over Flink on YARN cluster?

2019-08-26 Thread Yang Wang
Hi Qi, If you want to get better isolation between different flink jobs and multi-tenant support, i suggest you to use the per-job mode. Each flink job is a yarn application, and yarn use cgroup to limit the resource used by each application. Best, Yang Qi Kang 于2019年8月26日周一 下午9:02写道: > H

Re: How can TMs distribute evenly over Flink on YARN cluster?

2019-08-26 Thread Qi Kang
Hi Yang, Many thanks for your detailed explanation. We are using Hadoop 2.6.5, so setting multiple-assignments-enabled parameter is not an option. BTW, do you prefer using YARN session cluster rather than per-job cluster under this situation? These YARN nodes are almost dedicated to Flink job

Re: How can TMs distribute evenly over Flink on YARN cluster?

2019-08-26 Thread Yang Wang
Hi Qi Kang, If you means to spread out all taskmanager evenly across the yarn cluster, it is a pity that flink could do nothing. Each per-job flink cluster is an individual application on the yarn cluster, they do not know the existence of others. Could share the yarn version? If it is above hado

How can TMs distribute evenly over Flink on YARN cluster?

2019-08-26 Thread Qi Kang
Hi, We got 3 Flink jobs running on a 10-node YARN cluster. The jobs were submitted in a per-job flavor, with same parallelism (10) and number of slots per TM (2). We originally assumed that TMs should automatically spread across the cluster, but what came out was just the opposite: All 5 TMs

Re: Problem with Flink on Yarn

2019-08-23 Thread Rong Rong
r. > > Thanks, > Zhu Zhu > > Juan Gentile 于2019年8月23日周五 下午7:48写道: > >> Hello! >> >> >> >> We are running Flink on Yarn and we are currently getting the following >> error: >> >> >> >> *2019-08-23 06:11:01,534 WARN &g

Re: Problem with Flink on Yarn

2019-08-23 Thread Zhu Zhu
, Zhu Zhu Juan Gentile 于2019年8月23日周五 下午7:48写道: > Hello! > > > > We are running Flink on Yarn and we are currently getting the following > error: > > > > *2019-08-23 06:11:01,534 WARN > org.apache.hadoop.security.UserGroupInformation - >

Problem with Flink on Yarn

2019-08-23 Thread Juan Gentile
Hello! We are running Flink on Yarn and we are currently getting the following error: 2019-08-23 06:11:01,534 WARN org.apache.hadoop.security.UserGroupInformation - PriviledgedActionException as: (auth:KERBEROS) cause:org.apache.hadoop.ipc.RemoteException

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

Flink on YARN: TaskManager heap auto-sizing?

2019-05-07 Thread Dylan Adams
In the Configuration section of the docs , the description for "taskmanager.heap.size" contains: "On YARN setups, this value is automatically configured to the size of the TaskManager's YARN container, minus a

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

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 flin

flink on yarn log rolling

2019-03-20 Thread Shengnan YU
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 there still one "taskmanager.log" in yarn container log. Any idea with that? Thank you very much!

Re: Submitting job to Flink on yarn timesout on flip-6 1.5.x

2019-02-26 Thread Richard Deurwaarder
Hello Gary, Thank you for your response. I'd like to use the new mode but it does not work for me. It seems I am running into a firewall issue. Because the rest.port is random when running on yarn[1]. The machine I use to deploy the job can, in fact, start the Flink cluster, but it cannot submit

Re: Submitting job to Flink on yarn timesout on flip-6 1.5.x

2019-02-21 Thread Gary Yao
Hi, Beginning with Flink 1.7, you cannot use the legacy mode anymore [1][2]. I am currently working on removing references to the legacy mode in the documentation [3]. Is there any reason, you cannot use the "new mode"? Best, Gary [1] https://flink.apache.org/news/2018/11/30/release-1.7.0.html [

Submitting job to Flink on yarn timesout on flip-6 1.5.x

2019-02-18 Thread Richard Deurwaarder
Hello, I am trying to upgrade our job from flink 1.4.2 to 1.7.1 but I keep running into timeouts after submitting the job. The flink job runs on our hadoop cluster and starts using Yarn. Relevant config options seem to be: jobmanager.rpc.port: 55501 recovery.jobmanager.port: 55502 yarn.applic

Re: Running Flink on Yarn

2019-01-02 Thread Anil
Hi Andrey. Thanks for the reply. Apologies about the late follow up, I was out of office. Suppose I have 3 TM and each has 3 task slot and each kafka stream has 9 partitions each. Each thread will consumer from stream 1 (a1) and stream 2 (a2). Considering the query, data will need to be buffered

Re: Running Flink on Yarn

2018-12-24 Thread Andrey Zagrebin
I think the data buffered for join will be distributed among threads by order_id (a1 and a2 will be internally keyed). Each thread will have non-shared window state (for 2 hours) per certain order_id's. Slots will share some common JVM resources mentioned in docs, also access to state DB but not th

Re: Running Flink on Yarn

2018-12-24 Thread Anil
I am using time-windowed join only. Here's a sample query - SELECT a1.order_id, a2.order.restaurant_id FROM awz_s3_stream1 a1 INNER JOIN awz_s3_stream2 a2 ON CAST(a1.order_id AS VARCHAR) = a2.order_id AND a1.to_state = 'PLACED' AND a1.proctime BETWEEN a2.proctime - INTERVAL '2' HOUR AND a2.proct

Re: Running Flink on Yarn

2018-12-24 Thread Andrey Zagrebin
If you mean time-windowed join documented here [1]. I think it implicitly uses keyed stream [2] where the key is the field in equi-join predicate. The window state is also keyed [3] in this case. I also cc Timo and Piotr, they might add more to this topic. [1] https://ci.apache.org/projects/flink/

Re: Running Flink on Yarn

2018-12-24 Thread Anil
Thanks for the quick response Andrey. I'm doing a SQL time-windowed join on non-keyed stream. So all the thread in various task slot in the same TM will share this state. -- Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: Running Flink on Yarn

2018-12-24 Thread Andrey Zagrebin
Hi, I suppose you apply windowing to a keyed stream or SQL time-windowed join? Globally windowed streams are non-parallel and processed/stored in one slot. In case of keyed stream, total range of key values is distributed among slots. Each slot processes/stores only a subrange of keys. Window st

Running Flink on Yarn

2018-12-23 Thread Anil
I have a setup for Flink(1.4.2) with YARN. I'm using Flink Yarn Client for deploying my jobs to Yarn Cluster. In the current setup parallelism was directly mapped to the number of cores, with each parallel instance of the job running in one container. So for a parallelism of 9, there are 10 cont

Re: "405 HTTP method POST is not supported by this URL" is returned when POST a REST url on Flink on yarn

2018-10-30 Thread 徐涛
erything is OK > > "http://storage4.test.lan:8089/proxy/application_1532081306755_0124/jobs/269608367e0f548f30d98aa4efa2211e/checkpoints > > <http://storage4.test.lan:8089/proxy/application_1532081306755_0124/jobs/269608367e0f548f30d98aa4efa2211e/checkpoints>" > > &g

Re: "405 HTTP method POST is not supported by this URL" is returned when POST a REST url on Flink on yarn

2018-10-30 Thread 徐涛
storage4.test.lan:8089/proxy/application_1532081306755_0124/jobs/269608367e0f548f30d98aa4efa2211e/checkpoints > > <http://storage4.test.lan:8089/proxy/application_1532081306755_0124/jobs/269608367e0f548f30d98aa4efa2211e/checkpoints>" > > > Best > Henry > > >> 在 2

Re: Task manager count goes the expand then converge process when running flink on YARN

2018-10-25 Thread Till Rohrmann
Hi Henry, since version 1.5 you don't need to specify the number of TaskManagers to start, because the system will figure this out. Moreover, in version 1.5.x and 1.6.x it is recommended to set the number of slots per TaskManager to 1 since we did not support multi task slot TaskManagers properly.

Re: Task manager count goes the expand then converge process when running flink on YARN

2018-10-24 Thread vino yang
Hi Henry, The phenomenon you expressed is there, this is a bug, but I can't remember its JIRA number. Thanks, vino. 徐涛 于2018年10月24日周三 下午11:27写道: > Hi experts > I am running flink job on YARN in job cluster mode, the job is divided > into 2 tasks, the following are some configs of the job: > pa

Task manager count goes the expand then converge process when running flink on YARN

2018-10-24 Thread 徐涛
Hi experts I am running flink job on YARN in job cluster mode, the job is divided into 2 tasks, the following are some configs of the job: parallelism.default => 16 taskmanager.numberOfTaskSlots => 8 -yn => 2 when the program starts, I found that the count

Task manager count goes the expand then converge process when running flink on YARN

2018-10-24 Thread 徐涛
Hi experts I am running flink job on YARN in job cluster mode, the job is divided into 2 tasks, the following are some configs of the job: parallelism.default => 16 taskmanager.numberOfTaskSlots => 8 -yn => 2 when the program starts, I found that the count

  1   2   3   >