flink-user mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jain, Ankit" <ankit.j...@here.com>
Subject Re: High Availability on Yarn
Date Wed, 03 May 2017 14:58:50 GMT
Thanks for your reply Aljoscha.

After building better understanding of Yarn and spending copious amount of time on Flink codebase,
I think I now get how Flink & Yarn interact – I plan to document this soon in case it
could help somebody starting afresh with Flink-Yarn.

Regarding Zookeper, in YARN mode there is only one JobManager running, do we still need leader
election?

If the ApplicationMaster goes down (where JM runs) it is restarted by Yarn RM and while restarting,
Flink AM will bring back previous running containers.  So, where does Zookeeper sit in this
setup?

Thanks
Ankit

From: Aljoscha Krettek <aljoscha@apache.org>
Date: Wednesday, May 3, 2017 at 2:05 AM
To: "Jain, Ankit" <ankit.jain@here.com>
Cc: "user@flink.apache.org" <user@flink.apache.org>, Till Rohrmann <trohrmann@apache.org>
Subject: Re: High Availability on Yarn

Hi,
As a first comment, the work mentioned in the FLIP-6 doc you linked is still work-in-progress.
You cannot use these abstractions yet without going into the code and setting up a cluster
“by hand”.

The documentation for one-step deployment of a Job to YARN is available here: https://ci.apache.org/projects/flink/flink-docs-release-1.2/setup/yarn_setup.html#run-a-single-flink-job-on-yarn<https://emea01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fci.apache.org%2Fprojects%2Fflink%2Fflink-docs-release-1.2%2Fsetup%2Fyarn_setup.html%23run-a-single-flink-job-on-yarn&data=01%7C01%7C%7Cfb13823970ba4476ebbf08d49203846c%7C6d4034cd72254f72b85391feaea64919%7C1&sdata=fzBiQtv7MR2%2Fehg6GepwPa1uWxpqEgPJakto2B8k0Zk%3D&reserved=0>

Regarding your third question, ZooKeeper is mostly used for discovery and leader election.
That is, JobManagers use it to decide who is the main JM and who are standby JMs. TaskManagers
use it to discover the leading JobManager that they should connect to.

I’m also cc’ing Till, who should know this stuff better and can maybe explain it in a
bit more detail.

Best,
Aljoscha
On 1. May 2017, at 18:59, Jain, Ankit <ankit.jain@here.com<mailto:ankit.jain@here.com>>
wrote:

Hi fellow users,
We are trying to straighten out high availability story for flink.

Our setup includes a long running EMR cluster, job submission is a two-step process – 1)
Flink cluster is first created using flink yarn client on the EMR cluster already running
2) Flink job is submitted.

I also saw references that with 1.2, these two steps have been combined into 1 – is that
change in FlinkYarnSessionCli.java? Can somebody point to documentation please?

W/o worrying about Yarn RM (not Flink Yarn RM that seems to be newly introduced) failure for
now, I want to understand first how task manager & job manager failures are handled.

My questions-
1)       https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=65147077<https://emea01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fcwiki.apache.org%2Fconfluence%2Fpages%2Fviewpage.action%3FpageId%3D65147077&data=01%7C01%7C%7Cfb13823970ba4476ebbf08d49203846c%7C6d4034cd72254f72b85391feaea64919%7C1&sdata=29Se7mWQZ09ukF3rkQNmSRPXY4RkA8RCNO4ec4Glj8I%3D&reserved=0>
suggests a new RM has been added and now there is one JobManager for each job. Since Yarn
RM will now talk to Flink RM( instead of JobManager previously), will Yarn automatically restart
failing Flink RM?
2)       Is there any documentation on behavior of new Flink RM that will come up? How will
previously running JobManagers & TaskManagers find out about new RM?
3)       https://ci.apache.org/projects/flink/flink-docs-release-1.3/setup/jobmanager_high_availability.html#configuration<https://emea01.safelinks.protection.outlook.com/?url=https%3A%2F%2Fci.apache.org%2Fprojects%2Fflink%2Fflink-docs-release-1.3%2Fsetup%2Fjobmanager_high_availability.html%23configuration&data=01%7C01%7C%7Cfb13823970ba4476ebbf08d49203846c%7C6d4034cd72254f72b85391feaea64919%7C1&sdata=nYTYWaaWA4T1D7EwvL%2B7mwhrVcqn6xTzCv8SS6x%2FqLM%3D&reserved=0>
requires configuring Zookeeper even for Yarn – Is this needed for handling Task Manager
failures or JM or both? Will Yarn not take care of JM failures?

It may sound like I am little confused between role of Yarn and Flink components– who has
the most burden of HA? Documentation in current state is lacking clarity – I know it is
still evolving.

Please let me know if somebody can help clear the confusion.

Thanks
Ankit



Mime
View raw message