The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

classic Classic list List threaded Threaded
29 messages Options
12
Reply | Threaded
Open this post in threaded view
|

Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Till Rohrmann
Just a quick update. I've pushed a fixup which corrects a bug in the PR.

Cheers,
Till

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Yang Wang
Thanks till.

I have validated the PR, and it works as expected.
Future optimization will be discussed and done at FLINK-13184.

Best,
Yang

Till Rohrmann <[hidden email]> 于2019年11月6日周三 下午8:20写道:
Just a quick update. I've pushed a fixup which corrects a bug in the PR.

Cheers,
Till

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Chan, Regina

Yang and Till,

 

So I was able to test FLINK-12342 and it brought down the number of excess containers returned but it still doesn’t eliminate it. It’s just not as egregious as before. I uploaded a new file to the lockbox.

 

As for FLINK-13184, I tried applying the PR as it would look like if it were on top of the 1.9 code but then found something strange. Uploaded the log too. It would request the new containers log, that it received the containers and then start removing the container requests without starting execution of the job graph. Off hand, is there something else required from 1.10 for this to work as expected?

 

Thanks,

Regina

 

From: Yang Wang <[hidden email]>
Sent: Friday, November 8, 2019 6:55 AM
To: Till Rohrmann <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Thanks till.

 

I have validated the PR, and it works as expected.

Future optimization will be discussed and done at FLINK-13184.

 

Best,

Yang

 

Till Rohrmann <[hidden email]> 2019116日周三 下午8:20写道:

Just a quick update. I've pushed a fixup which corrects a bug in the PR.

 

Cheers,

Till

 

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Yang Wang
Hi Regina,

Sorry for late. Do you mean after PR-10089 you still get excess containers? I have explained before, the excess
container could not be eliminated in any time. If some containers were allocated between the AM-YarnRM heartbeat,
we could get some excess containers. It is just because the Yarn allocation mechanism. However, the excess
containers should be returned immediately and the amount should not be big.

For Flink-13184, we want to reduce the excess container further on through avoid the blocking operation in 
ResourceManager main thread. Since the code base has changed, i will attach another PR for release-1.9.

I will check your log to verify whether it is expected behavior.

Best,
Yang

Chan, Regina <[hidden email]> 于2019年11月15日周五 上午6:29写道:

Yang and Till,

 

So I was able to test FLINK-12342 and it brought down the number of excess containers returned but it still doesn’t eliminate it. It’s just not as egregious as before. I uploaded a new file to the lockbox.

 

As for FLINK-13184, I tried applying the PR as it would look like if it were on top of the 1.9 code but then found something strange. Uploaded the log too. It would request the new containers log, that it received the containers and then start removing the container requests without starting execution of the job graph. Off hand, is there something else required from 1.10 for this to work as expected?

 

Thanks,

Regina

 

From: Yang Wang <[hidden email]>
Sent: Friday, November 8, 2019 6:55 AM
To: Till Rohrmann <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Thanks till.

 

I have validated the PR, and it works as expected.

Future optimization will be discussed and done at FLINK-13184.

 

Best,

Yang

 

Till Rohrmann <[hidden email]> 2019116日周三 下午8:20写道:

Just a quick update. I've pushed a fixup which corrects a bug in the PR.

 

Cheers,

Till

 

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Chan, Regina

Hi Yang,

 

Right. I’m just confirming that PR-10089 brought the number of excess containers. Thanks Till.

 

I was hoping to get FLINK-13184 tested also but when I applied the changes onto 1.9, it seems to be broken as it receives the containers and then releases them without ever running workload on the containers. I attached the logs with that behavior. My changes are exactly the changes in the 1.9-PR.

 

 

From: Yang Wang <[hidden email]>
Sent: Tuesday, November 19, 2019 3:00 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

Sorry for late. Do you mean after PR-10089 you still get excess containers? I have explained before, the excess

container could not be eliminated in any time. If some containers were allocated between the AM-YarnRM heartbeat,

we could get some excess containers. It is just because the Yarn allocation mechanism. However, the excess

containers should be returned immediately and the amount should not be big.

 

For Flink-13184, we want to reduce the excess container further on through avoid the blocking operation in 

ResourceManager main thread. Since the code base has changed, i will attach another PR for release-1.9.

 

I will check your log to verify whether it is expected behavior.

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191115日周五 上午6:29写道:

Yang and Till,

 

So I was able to test FLINK-12342 and it brought down the number of excess containers returned but it still doesn’t eliminate it. It’s just not as egregious as before. I uploaded a new file to the lockbox.

 

As for FLINK-13184, I tried applying the PR as it would look like if it were on top of the 1.9 code but then found something strange. Uploaded the log too. It would request the new containers log, that it received the containers and then start removing the container requests without starting execution of the job graph. Off hand, is there something else required from 1.10 for this to work as expected?

 

Thanks,

Regina

 

From: Yang Wang <[hidden email]>
Sent: Friday, November 8, 2019 6:55 AM
To: Till Rohrmann <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Thanks till.

 

I have validated the PR, and it works as expected.

Future optimization will be discussed and done at FLINK-13184.

 

Best,

Yang

 

Till Rohrmann <[hidden email]> 2019116日周三 下午8:20写道:

Just a quick update. I've pushed a fixup which corrects a bug in the PR.

 

Cheers,

Till

 

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Chan, Regina
In reply to this post by Yang Wang

Till,

 

This is something new – I’ll provide the full logs in the lockbox. Can you help take a look?

 

Even for small containers ie 1 TM, 2 slots in this case, what happens is that it gets the containers, but then releases them right away. Finally the after doing this for about 5 minutes, it finally gets a NoResourceAvailableException. This is the code with PR-10089 only.

 

2019-11-25 21:05:52,212 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Registration at ResourceManager attempt 1 (timeout=100ms)

2019-11-25 21:05:52,214 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registering job manager [hidden email]://[hidden email]:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,219 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registered job manager [hidden email]://[hidden email]:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,221 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - JobManager successfully registered at ResourceManager, leader id: 00000000000000000000000000000000.

2019-11-25 21:05:52,222 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Requesting new slot [SlotRequestId{e1bc173469db5c037024bc112ea7cf27}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} from resource manager.

2019-11-25 21:05:52,225 INFO  org.apache.flink.yarn.YarnResourceManager                     - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} for job 18fc10b01779ada49f0738fa4c4e35f7 with allocation id f007f6d3b8b110dce8200c68b4e82e11.

2019-11-25 21:05:52,232 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:05:56,282 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-347.dc.gs.com:45454

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:05:56,817 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:05:56,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:05:56,832 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-347.dc.gs.com:45454

2019-11-25 21:06:02,293 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:07,797 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-147.dc.gs.com:45454

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:08,639 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:08,640 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:08,642 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-147.dc.gs.com:45454

2019-11-25 21:06:18,805 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:24,309 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-240.dc.gs.com:45454

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:24,664 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:24,665 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:24,668 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-240.dc.gs.com:45454

2019-11-25 21:06:34,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:40,322 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-301.dc.gs.com:45454

2019-11-25 21:06:40,322 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:40,323 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:41,545 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:41,546 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:41,548 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-301.dc.gs.com:45454

2019-11-25 21:06:46,837 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:52,340 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-152.dc.gs.com:45454

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:52,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:52,613 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

 

<more of these>

 

2019-11-25 21:10:37,823 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-255.dc.gs.com:45454

2019-11-25 21:10:43,108 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:10:48,610 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-365.dc.gs.com:45454

2019-11-25 21:10:48,611 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:10:48,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:10:48,817 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-365.dc.gs.com:45454

2019-11-25 21:10:52,194 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (exfd_daily_fund_prices | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Filter (Filter at validateData(DAXTask.java:68)) -> FlatMap (FlatMap at handleBloomFilter(PreMergeTask.java:168)) -> FlatMap (FlatMap at collapsePipelineIfRequired(Task.java:160)) (1/1) (38f233581fbbe385097692e3cc91b57c) switched from SCHEDULED to FAILED.

org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.

        at org.apache.flink.runtime.executiongraph.Execution.lambda$scheduleForExecution$0(Execution.java:460)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl.lambda$internalAllocateSlot$0(SchedulerImpl.java:190)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:700)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:484)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.lambda$new$0(SlotSharingManager.java:380)

        at java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)

        at java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:998)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)

        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)

        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at akka.actor.Actor$class.aroundReceive(Actor.scala:517)

        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)

        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)

        at akka.actor.ActorCell.invoke(ActorCell.scala:561)

        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)

        at akka.dispatch.Mailbox.run(Mailbox.scala:225)

        at akka.dispatch.Mailbox.exec(Mailbox.scala:235)

        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

        at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

        at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, November 20, 2019 4:05 PM
To: 'Yang Wang' <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Yang,

 

Right. I’m just confirming that PR-10089 brought the number of excess containers. Thanks Till.

 

I was hoping to get FLINK-13184 tested also but when I applied the changes onto 1.9, it seems to be broken as it receives the containers and then releases them without ever running workload on the containers. I attached the logs with that behavior. My changes are exactly the changes in the 1.9-PR.

 

 

From: Yang Wang <[hidden email]>
Sent: Tuesday, November 19, 2019 3:00 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

Sorry for late. Do you mean after PR-10089 you still get excess containers? I have explained before, the excess

container could not be eliminated in any time. If some containers were allocated between the AM-YarnRM heartbeat,

we could get some excess containers. It is just because the Yarn allocation mechanism. However, the excess

containers should be returned immediately and the amount should not be big.

 

For Flink-13184, we want to reduce the excess container further on through avoid the blocking operation in 

ResourceManager main thread. Since the code base has changed, i will attach another PR for release-1.9.

 

I will check your log to verify whether it is expected behavior.

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191115日周五 上午6:29写道:

Yang and Till,

 

So I was able to test FLINK-12342 and it brought down the number of excess containers returned but it still doesn’t eliminate it. It’s just not as egregious as before. I uploaded a new file to the lockbox.

 

As for FLINK-13184, I tried applying the PR as it would look like if it were on top of the 1.9 code but then found something strange. Uploaded the log too. It would request the new containers log, that it received the containers and then start removing the container requests without starting execution of the job graph. Off hand, is there something else required from 1.10 for this to work as expected?

 

Thanks,

Regina

 

From: Yang Wang <[hidden email]>
Sent: Friday, November 8, 2019 6:55 AM
To: Till Rohrmann <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Thanks till.

 

I have validated the PR, and it works as expected.

Future optimization will be discussed and done at FLINK-13184.

 

Best,

Yang

 

Till Rohrmann <[hidden email]> 2019116日周三 下午8:20写道:

Just a quick update. I've pushed a fixup which corrects a bug in the PR.

 

Cheers,

Till

 

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Till Rohrmann
Hi Regina,

it does not look that Flink stops the containers because otherwise we would see "Stopping container ..." in the logs. Instead I suspect that the TM startup on Yarn somehow failed. The first step to further understand the problem would be to enable the DEBUG log level. Then you could also take a look into the Yarn logs to see whether they contain something suspicious.

FLINK-13184 should now also be merged in the release-1.9 branch. If you check this branch out and build Flink from there, you should be able to try this feature out.

Cheers,
Till

On Tue, Nov 26, 2019 at 11:35 PM Chan, Regina <[hidden email]> wrote:

Till,

 

This is something new – I’ll provide the full logs in the lockbox. Can you help take a look?

 

Even for small containers ie 1 TM, 2 slots in this case, what happens is that it gets the containers, but then releases them right away. Finally the after doing this for about 5 minutes, it finally gets a NoResourceAvailableException. This is the code with PR-10089 only.

 

2019-11-25 21:05:52,212 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Registration at ResourceManager attempt 1 (timeout=100ms)

2019-11-25 21:05:52,214 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registering job manager [hidden email]://flink@...:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,219 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registered job manager [hidden email]://flink@...:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,221 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - JobManager successfully registered at ResourceManager, leader id: 00000000000000000000000000000000.

2019-11-25 21:05:52,222 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Requesting new slot [SlotRequestId{e1bc173469db5c037024bc112ea7cf27}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} from resource manager.

2019-11-25 21:05:52,225 INFO  org.apache.flink.yarn.YarnResourceManager                     - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} for job 18fc10b01779ada49f0738fa4c4e35f7 with allocation id f007f6d3b8b110dce8200c68b4e82e11.

2019-11-25 21:05:52,232 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:05:56,282 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-347.dc.gs.com:45454

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:05:56,817 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:05:56,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:05:56,832 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-347.dc.gs.com:45454

2019-11-25 21:06:02,293 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:07,797 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-147.dc.gs.com:45454

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:08,639 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:08,640 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:08,642 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-147.dc.gs.com:45454

2019-11-25 21:06:18,805 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:24,309 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-240.dc.gs.com:45454

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:24,664 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:24,665 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:24,668 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-240.dc.gs.com:45454

2019-11-25 21:06:34,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:40,322 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-301.dc.gs.com:45454

2019-11-25 21:06:40,322 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:40,323 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:41,545 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:41,546 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:41,548 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-301.dc.gs.com:45454

2019-11-25 21:06:46,837 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:52,340 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-152.dc.gs.com:45454

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:52,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:52,613 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

 

<more of these>

 

2019-11-25 21:10:37,823 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-255.dc.gs.com:45454

2019-11-25 21:10:43,108 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:10:48,610 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-365.dc.gs.com:45454

2019-11-25 21:10:48,611 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:10:48,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:10:48,817 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-365.dc.gs.com:45454

2019-11-25 21:10:52,194 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (exfd_daily_fund_prices | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Filter (Filter at validateData(DAXTask.java:68)) -> FlatMap (FlatMap at handleBloomFilter(PreMergeTask.java:168)) -> FlatMap (FlatMap at collapsePipelineIfRequired(Task.java:160)) (1/1) (38f233581fbbe385097692e3cc91b57c) switched from SCHEDULED to FAILED.

org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.

        at org.apache.flink.runtime.executiongraph.Execution.lambda$scheduleForExecution$0(Execution.java:460)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl.lambda$internalAllocateSlot$0(SchedulerImpl.java:190)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:700)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:484)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.lambda$new$0(SlotSharingManager.java:380)

        at java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)

        at java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:998)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)

        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)

        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at akka.actor.Actor$class.aroundReceive(Actor.scala:517)

        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)

        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)

        at akka.actor.ActorCell.invoke(ActorCell.scala:561)

        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)

        at akka.dispatch.Mailbox.run(Mailbox.scala:225)

        at akka.dispatch.Mailbox.exec(Mailbox.scala:235)

        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

        at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

        at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, November 20, 2019 4:05 PM
To: 'Yang Wang' <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Yang,

 

Right. I’m just confirming that PR-10089 brought the number of excess containers. Thanks Till.

 

I was hoping to get FLINK-13184 tested also but when I applied the changes onto 1.9, it seems to be broken as it receives the containers and then releases them without ever running workload on the containers. I attached the logs with that behavior. My changes are exactly the changes in the 1.9-PR.

 

 

From: Yang Wang <[hidden email]>
Sent: Tuesday, November 19, 2019 3:00 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

Sorry for late. Do you mean after PR-10089 you still get excess containers? I have explained before, the excess

container could not be eliminated in any time. If some containers were allocated between the AM-YarnRM heartbeat,

we could get some excess containers. It is just because the Yarn allocation mechanism. However, the excess

containers should be returned immediately and the amount should not be big.

 

For Flink-13184, we want to reduce the excess container further on through avoid the blocking operation in 

ResourceManager main thread. Since the code base has changed, i will attach another PR for release-1.9.

 

I will check your log to verify whether it is expected behavior.

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191115日周五 上午6:29写道:

Yang and Till,

 

So I was able to test FLINK-12342 and it brought down the number of excess containers returned but it still doesn’t eliminate it. It’s just not as egregious as before. I uploaded a new file to the lockbox.

 

As for FLINK-13184, I tried applying the PR as it would look like if it were on top of the 1.9 code but then found something strange. Uploaded the log too. It would request the new containers log, that it received the containers and then start removing the container requests without starting execution of the job graph. Off hand, is there something else required from 1.10 for this to work as expected?

 

Thanks,

Regina

 

From: Yang Wang <[hidden email]>
Sent: Friday, November 8, 2019 6:55 AM
To: Till Rohrmann <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Thanks till.

 

I have validated the PR, and it works as expected.

Future optimization will be discussed and done at FLINK-13184.

 

Best,

Yang

 

Till Rohrmann <[hidden email]> 2019116日周三 下午8:20写道:

Just a quick update. I've pushed a fixup which corrects a bug in the PR.

 

Cheers,

Till

 

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Hailu, Andreas

Hi Till,

 

Since the last time we’ve reached out, we’ve upgraded our codebase to Flink 1.9.1 and are now trying out FLINK-13184 ahead of the 1.9.2 release with the hopes of stabilizing our widest flow.

 

Before we patched in FLINK-13184, we saw failures with the RegistrationTimeoutException pattern linked in the ticket. With the patch, the behavior we observe is a series of jobs get submitted, and then fail with a NoResourceAvailable exception after the default 5 minute timeout. I see a good amount of the following pattern:

 

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner           - JobManager runner for job Flink Java Job at Wed Jan 29 10:43:45 EST 2020 (a7dae17c1c43821d7993e8fb6523cad3) was granted leadership with session id 00000000-0000-0000-0000-000000000000 at akka.tcp://[hidden email]:46075/user/jobmanager_57.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Starting execution of job Flink Java Job at Wed Jan 29 10:43:45 EST 2020 (a7dae17c1c43821d7993e8fb6523cad3) under job master id 00000000000000000000000000000000.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job Flink Java Job at Wed Jan 29 10:43:45 EST 2020 (a7dae17c1c43821d7993e8fb6523cad3) switched from state CREATED to RUNNING.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (1/5) (249df12d45cad3fa34622e545c1fff99) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{316a5e101c5aa27380017596715f198d}]

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (2/5) (e8275774afd97670bfd71d49ac464691) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{156388b9833814b76b90bb5083815503}]

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (3/5) (50702de9bf21ccfc48c9b9b17f9d9d2b) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{c3544184df837ebfe2a5091b6f44e6c4}]

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (4/5) (b6f6e78c1b4711c79c91203a66aa83ed) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{91c2170a57e6ed545fce67c2111d7226}]

2020-01-29 10:43:57,115 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (5/5) (f2a19c8d536a8e1f87fb37e6cebd1923) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,115 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{2f142da8cc259ce2ab34edab1e877a8d}]

 

In comparison, the patch 13184 seems to take longer to acquire containers, and there’s some odd behavior in the logs. We see receptions of a small quantity of containers:

 

2020-01-29 10:42:22,318 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 128 pending container requests.

2020-01-29 10:42:22,319 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 127.

...

2020-01-29 10:42:52,317 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 2 containers with 132 pending container requests.

2020-01-29 10:42:52,318 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 131.

2020-01-29 10:42:52,318 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 130.

...

2020-01-29 10:43:22,327 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 130 pending container requests.

2020-01-29 10:43:22,327 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 129

...

2020-01-29 10:46:22,411 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 129 pending container requests.

2020-01-29 10:46:22,411 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 1

...

2020-01-29 11:01:53,314 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 135 pending container requests.

2020-01-29 11:01:53,314 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 134

 

And then later down the line, we receive a ton:

2020-01-29 11:13:23,810 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 80 containers with 125 pending container requests.

...

2020-01-29 11:15:23,937 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 45 containers with 60 pending container requests.

...

2020-01-29 11:15:53,965 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 40 containers with 15 pending container requests.

 

I’m unsure if this is directly related, though. It almost seems like with 13184, it takes longer to acquire containers. Is it possible that we’re just missing other features from 1.9.2? I’ve uploaded these logs to the same Lockbox that Regina shared with you last November - it’s the file with 13184 in the name. Could you help take a look?

 

Best,

Andreas

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, December 3, 2019 6:12 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

it does not look that Flink stops the containers because otherwise we would see "Stopping container ..." in the logs. Instead I suspect that the TM startup on Yarn somehow failed. The first step to further understand the problem would be to enable the DEBUG log level. Then you could also take a look into the Yarn logs to see whether they contain something suspicious.

 

FLINK-13184 should now also be merged in the release-1.9 branch. If you check this branch out and build Flink from there, you should be able to try this feature out.

 

Cheers,

Till

 

On Tue, Nov 26, 2019 at 11:35 PM Chan, Regina <[hidden email]> wrote:

Till,

 

This is something new – I’ll provide the full logs in the lockbox. Can you help take a look?

 

Even for small containers ie 1 TM, 2 slots in this case, what happens is that it gets the containers, but then releases them right away. Finally the after doing this for about 5 minutes, it finally gets a NoResourceAvailableException. This is the code with PR-10089 only.

 

2019-11-25 21:05:52,212 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Registration at ResourceManager attempt 1 (timeout=100ms)

2019-11-25 21:05:52,214 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registering job manager [hidden email]://flink@...:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,219 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registered job manager [hidden email]://flink@...:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,221 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - JobManager successfully registered at ResourceManager, leader id: 00000000000000000000000000000000.

2019-11-25 21:05:52,222 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Requesting new slot [SlotRequestId{e1bc173469db5c037024bc112ea7cf27}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} from resource manager.

2019-11-25 21:05:52,225 INFO  org.apache.flink.yarn.YarnResourceManager                     - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} for job 18fc10b01779ada49f0738fa4c4e35f7 with allocation id f007f6d3b8b110dce8200c68b4e82e11.

2019-11-25 21:05:52,232 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:05:56,282 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-347.dc.gs.com:45454

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:05:56,817 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:05:56,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:05:56,832 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-347.dc.gs.com:45454

2019-11-25 21:06:02,293 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:07,797 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-147.dc.gs.com:45454

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:08,639 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:08,640 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:08,642 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-147.dc.gs.com:45454

2019-11-25 21:06:18,805 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:24,309 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-240.dc.gs.com:45454

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:24,664 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:24,665 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:24,668 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-240.dc.gs.com:45454

2019-11-25 21:06:34,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:40,322 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-301.dc.gs.com:45454

2019-11-25 21:06:40,322 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:40,323 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:41,545 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:41,546 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:41,548 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-301.dc.gs.com:45454

2019-11-25 21:06:46,837 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:52,340 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-152.dc.gs.com:45454

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:52,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:52,613 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

 

<more of these>

 

2019-11-25 21:10:37,823 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-255.dc.gs.com:45454

2019-11-25 21:10:43,108 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:10:48,610 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-365.dc.gs.com:45454

2019-11-25 21:10:48,611 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:10:48,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:10:48,817 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-365.dc.gs.com:45454

2019-11-25 21:10:52,194 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (exfd_daily_fund_prices | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Filter (Filter at validateData(DAXTask.java:68)) -> FlatMap (FlatMap at handleBloomFilter(PreMergeTask.java:168)) -> FlatMap (FlatMap at collapsePipelineIfRequired(Task.java:160)) (1/1) (38f233581fbbe385097692e3cc91b57c) switched from SCHEDULED to FAILED.

org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.

        at org.apache.flink.runtime.executiongraph.Execution.lambda$scheduleForExecution$0(Execution.java:460)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl.lambda$internalAllocateSlot$0(SchedulerImpl.java:190)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:700)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:484)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.lambda$new$0(SlotSharingManager.java:380)

        at java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)

        at java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:998)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)

        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)

        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at akka.actor.Actor$class.aroundReceive(Actor.scala:517)

        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)

        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)

        at akka.actor.ActorCell.invoke(ActorCell.scala:561)

        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)

        at akka.dispatch.Mailbox.run(Mailbox.scala:225)

        at akka.dispatch.Mailbox.exec(Mailbox.scala:235)

        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

        at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

        at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, November 20, 2019 4:05 PM
To: 'Yang Wang' <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Yang,

 

Right. I’m just confirming that PR-10089 brought the number of excess containers. Thanks Till.

 

I was hoping to get FLINK-13184 tested also but when I applied the changes onto 1.9, it seems to be broken as it receives the containers and then releases them without ever running workload on the containers. I attached the logs with that behavior. My changes are exactly the changes in the 1.9-PR.

 

 

From: Yang Wang <[hidden email]>
Sent: Tuesday, November 19, 2019 3:00 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

Sorry for late. Do you mean after PR-10089 you still get excess containers? I have explained before, the excess

container could not be eliminated in any time. If some containers were allocated between the AM-YarnRM heartbeat,

we could get some excess containers. It is just because the Yarn allocation mechanism. However, the excess

containers should be returned immediately and the amount should not be big.

 

For Flink-13184, we want to reduce the excess container further on through avoid the blocking operation in 

ResourceManager main thread. Since the code base has changed, i will attach another PR for release-1.9.

 

I will check your log to verify whether it is expected behavior.

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191115日周五 上午6:29写道:

Yang and Till,

 

So I was able to test FLINK-12342 and it brought down the number of excess containers returned but it still doesn’t eliminate it. It’s just not as egregious as before. I uploaded a new file to the lockbox.

 

As for FLINK-13184, I tried applying the PR as it would look like if it were on top of the 1.9 code but then found something strange. Uploaded the log too. It would request the new containers log, that it received the containers and then start removing the container requests without starting execution of the job graph. Off hand, is there something else required from 1.10 for this to work as expected?

 

Thanks,

Regina

 

From: Yang Wang <[hidden email]>
Sent: Friday, November 8, 2019 6:55 AM
To: Till Rohrmann <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Thanks till.

 

I have validated the PR, and it works as expected.

Future optimization will be discussed and done at FLINK-13184.

 

Best,

Yang

 

Till Rohrmann <[hidden email]> 2019116日周三 下午8:20写道:

Just a quick update. I've pushed a fixup which corrects a bug in the PR.

 

Cheers,

Till

 

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
Reply | Threaded
Open this post in threaded view
|

Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

Till Rohrmann
Hi Andreas,

the effect of FLINK-13184 should be that we no longer ask for too many containers but not that it takes longer to fulfil the requests. I would suggest that you take a look at the Yarn logs to see whether there is anything suspicious (maybe the resource capacities of the queue you are deploying to is depleted). 

Maybe Yarn was fulfilling requests faster wrt other applications if there were more requests relatively to the other applications. Maybe the scheduling strategy of your Yarn cluster has been changed from fifo to fair, for example.

I'll try to take a look at the logs to see whether I can find anything suspicious.

Cheers,
Till

On Wed, Jan 29, 2020 at 9:04 PM Hailu, Andreas <[hidden email]> wrote:

Hi Till,

 

Since the last time we’ve reached out, we’ve upgraded our codebase to Flink 1.9.1 and are now trying out FLINK-13184 ahead of the 1.9.2 release with the hopes of stabilizing our widest flow.

 

Before we patched in FLINK-13184, we saw failures with the RegistrationTimeoutException pattern linked in the ticket. With the patch, the behavior we observe is a series of jobs get submitted, and then fail with a NoResourceAvailable exception after the default 5 minute timeout. I see a good amount of the following pattern:

 

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.JobManagerRunner           - JobManager runner for job Flink Java Job at Wed Jan 29 10:43:45 EST 2020 (a7dae17c1c43821d7993e8fb6523cad3) was granted leadership with session id 00000000-0000-0000-0000-000000000000 at akka.tcp://flink@...:46075/user/jobmanager_57.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Starting execution of job Flink Java Job at Wed Jan 29 10:43:45 EST 2020 (a7dae17c1c43821d7993e8fb6523cad3) under job master id 00000000000000000000000000000000.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Job Flink Java Job at Wed Jan 29 10:43:45 EST 2020 (a7dae17c1c43821d7993e8fb6523cad3) switched from state CREATED to RUNNING.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (1/5) (249df12d45cad3fa34622e545c1fff99) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{316a5e101c5aa27380017596715f198d}]

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (2/5) (e8275774afd97670bfd71d49ac464691) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{156388b9833814b76b90bb5083815503}]

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (3/5) (50702de9bf21ccfc48c9b9b17f9d9d2b) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{c3544184df837ebfe2a5091b6f44e6c4}]

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (4/5) (b6f6e78c1b4711c79c91203a66aa83ed) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,114 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{91c2170a57e6ed545fce67c2111d7226}]

2020-01-29 10:43:57,115 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (entity_product_details | Read Merged Data From File System | PARQUET) -> Filter (entity_product_details | Filter Live Records | PARQUET) -> Map (Map at readParquetFileWithSchema(FlinkReadUtils.java:142)) -> Map (Key Extractor) (5/5) (f2a19c8d536a8e1f87fb37e6cebd1923) switched from CREATED to SCHEDULED.

2020-01-29 10:43:57,115 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Cannot serve slot request, no ResourceManager connected. Adding as pending request [SlotRequestId{2f142da8cc259ce2ab34edab1e877a8d}]

 

In comparison, the patch 13184 seems to take longer to acquire containers, and there’s some odd behavior in the logs. We see receptions of a small quantity of containers:

 

2020-01-29 10:42:22,318 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 128 pending container requests.

2020-01-29 10:42:22,319 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 127.

...

2020-01-29 10:42:52,317 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 2 containers with 132 pending container requests.

2020-01-29 10:42:52,318 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 131.

2020-01-29 10:42:52,318 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 130.

...

2020-01-29 10:43:22,327 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 130 pending container requests.

2020-01-29 10:43:22,327 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 129

...

2020-01-29 10:46:22,411 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 129 pending container requests.

2020-01-29 10:46:22,411 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 1

...

2020-01-29 11:01:53,314 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 135 pending container requests.

2020-01-29 11:01:53,314 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:20480, vCores:1>]Priority[1]. Pending container requests 134

 

And then later down the line, we receive a ton:

2020-01-29 11:13:23,810 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 80 containers with 125 pending container requests.

...

2020-01-29 11:15:23,937 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 45 containers with 60 pending container requests.

...

2020-01-29 11:15:53,965 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 40 containers with 15 pending container requests.

 

I’m unsure if this is directly related, though. It almost seems like with 13184, it takes longer to acquire containers. Is it possible that we’re just missing other features from 1.9.2? I’ve uploaded these logs to the same Lockbox that Regina shared with you last November - it’s the file with 13184 in the name. Could you help take a look?

 

Best,

Andreas

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, December 3, 2019 6:12 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

it does not look that Flink stops the containers because otherwise we would see "Stopping container ..." in the logs. Instead I suspect that the TM startup on Yarn somehow failed. The first step to further understand the problem would be to enable the DEBUG log level. Then you could also take a look into the Yarn logs to see whether they contain something suspicious.

 

FLINK-13184 should now also be merged in the release-1.9 branch. If you check this branch out and build Flink from there, you should be able to try this feature out.

 

Cheers,

Till

 

On Tue, Nov 26, 2019 at 11:35 PM Chan, Regina <[hidden email]> wrote:

Till,

 

This is something new – I’ll provide the full logs in the lockbox. Can you help take a look?

 

Even for small containers ie 1 TM, 2 slots in this case, what happens is that it gets the containers, but then releases them right away. Finally the after doing this for about 5 minutes, it finally gets a NoResourceAvailableException. This is the code with PR-10089 only.

 

2019-11-25 21:05:52,212 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - Registration at ResourceManager attempt 1 (timeout=100ms)

2019-11-25 21:05:52,214 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registering job manager [hidden email]://flink@...:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,219 INFO  org.apache.flink.yarn.YarnResourceManager                     - Registered job manager [hidden email]://flink@...:35041/user/jobmanager_0 for job 18fc10b01779ada49f0738fa4c4e35f7.

2019-11-25 21:05:52,221 INFO  org.apache.flink.runtime.jobmaster.JobMaster                  - JobManager successfully registered at ResourceManager, leader id: 00000000000000000000000000000000.

2019-11-25 21:05:52,222 INFO  org.apache.flink.runtime.jobmaster.slotpool.SlotPoolImpl      - Requesting new slot [SlotRequestId{e1bc173469db5c037024bc112ea7cf27}] and profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} from resource manager.

2019-11-25 21:05:52,225 INFO  org.apache.flink.yarn.YarnResourceManager                     - Request slot with profile ResourceProfile{cpuCores=-1.0, heapMemoryInMB=-1, directMemoryInMB=-1, nativeMemoryInMB=-1, networkMemoryInMB=-1, managedMemoryInMB=-1} for job 18fc10b01779ada49f0738fa4c4e35f7 with allocation id f007f6d3b8b110dce8200c68b4e82e11.

2019-11-25 21:05:52,232 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:05:56,282 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-347.dc.gs.com:45454

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:05:56,287 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:05:56,817 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:05:56,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:05:56,832 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-347.dc.gs.com:45454

2019-11-25 21:06:02,293 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:07,797 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-147.dc.gs.com:45454

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:07,798 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:08,639 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:08,640 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:08,642 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-147.dc.gs.com:45454

2019-11-25 21:06:18,805 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:24,309 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-240.dc.gs.com:45454

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:24,310 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:24,664 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:24,665 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:24,668 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-240.dc.gs.com:45454

2019-11-25 21:06:34,818 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:40,322 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-301.dc.gs.com:45454

2019-11-25 21:06:40,322 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:40,323 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:41,545 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:41,546 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:06:41,548 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-301.dc.gs.com:45454

2019-11-25 21:06:46,837 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:06:52,340 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-152.dc.gs.com:45454

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:06:52,341 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:06:52,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:06:52,613 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

 

<more of these>

 

2019-11-25 21:10:37,823 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-255.dc.gs.com:45454

2019-11-25 21:10:43,108 INFO  org.apache.flink.yarn.YarnResourceManager                     - Requesting new TaskExecutor container with resources <memory:12288, vCores:2>. Number pending requests 1.

2019-11-25 21:10:48,610 INFO  org.apache.hadoop.yarn.client.api.impl.AMRMClientImpl         - Received new token for : d73996-365.dc.gs.com:45454

2019-11-25 21:10:48,611 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received 1 containers with 1 pending container requests.

2019-11-25 21:10:48,612 INFO  org.apache.flink.yarn.YarnResourceManager                     - Removing container request Capability[<memory:12288, vCores:2>]Priority[1]. Pending container requests 0.

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Creating container launch context for TaskManagers

2019-11-25 21:10:48,815 INFO  org.apache.flink.yarn.YarnResourceManager                     - Starting TaskManagers

2019-11-25 21:10:48,817 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d73996-365.dc.gs.com:45454

2019-11-25 21:10:52,194 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - CHAIN DataSource (exfd_daily_fund_prices | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Filter (Filter at validateData(DAXTask.java:68)) -> FlatMap (FlatMap at handleBloomFilter(PreMergeTask.java:168)) -> FlatMap (FlatMap at collapsePipelineIfRequired(Task.java:160)) (1/1) (38f233581fbbe385097692e3cc91b57c) switched from SCHEDULED to FAILED.

org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.

        at org.apache.flink.runtime.executiongraph.Execution.lambda$scheduleForExecution$0(Execution.java:460)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SchedulerImpl.lambda$internalAllocateSlot$0(SchedulerImpl.java:190)

        at java.util.concurrent.CompletableFuture.uniWhenComplete(CompletableFuture.java:760)

        at java.util.concurrent.CompletableFuture$UniWhenComplete.tryFire(CompletableFuture.java:736)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$SingleTaskSlot.release(SlotSharingManager.java:700)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.release(SlotSharingManager.java:484)

        at org.apache.flink.runtime.jobmaster.slotpool.SlotSharingManager$MultiTaskSlot.lambda$new$0(SlotSharingManager.java:380)

        at java.util.concurrent.CompletableFuture.uniHandle(CompletableFuture.java:822)

        at java.util.concurrent.CompletableFuture$UniHandle.tryFire(CompletableFuture.java:797)

        at java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)

        at java.util.concurrent.CompletableFuture.completeExceptionally(CompletableFuture.java:1977)

        at org.apache.flink.runtime.concurrent.FutureUtils$Timeout.run(FutureUtils.java:998)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRunAsync(AkkaRpcActor.java:397)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleRpcMessage(AkkaRpcActor.java:190)

        at org.apache.flink.runtime.rpc.akka.FencedAkkaRpcActor.handleRpcMessage(FencedAkkaRpcActor.java:74)

        at org.apache.flink.runtime.rpc.akka.AkkaRpcActor.handleMessage(AkkaRpcActor.java:152)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:26)

        at akka.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:21)

        at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)

        at akka.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:21)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:170)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:171)

        at akka.actor.Actor$class.aroundReceive(Actor.scala:517)

        at akka.actor.AbstractActor.aroundReceive(AbstractActor.scala:225)

        at akka.actor.ActorCell.receiveMessage(ActorCell.scala:592)

        at akka.actor.ActorCell.invoke(ActorCell.scala:561)

        at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:258)

        at akka.dispatch.Mailbox.run(Mailbox.scala:225)

        at akka.dispatch.Mailbox.exec(Mailbox.scala:235)

        at akka.dispatch.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

        at akka.dispatch.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

        at akka.dispatch.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, November 20, 2019 4:05 PM
To: 'Yang Wang' <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Yang,

 

Right. I’m just confirming that PR-10089 brought the number of excess containers. Thanks Till.

 

I was hoping to get FLINK-13184 tested also but when I applied the changes onto 1.9, it seems to be broken as it receives the containers and then releases them without ever running workload on the containers. I attached the logs with that behavior. My changes are exactly the changes in the 1.9-PR.

 

 

From: Yang Wang <[hidden email]>
Sent: Tuesday, November 19, 2019 3:00 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

Sorry for late. Do you mean after PR-10089 you still get excess containers? I have explained before, the excess

container could not be eliminated in any time. If some containers were allocated between the AM-YarnRM heartbeat,

we could get some excess containers. It is just because the Yarn allocation mechanism. However, the excess

containers should be returned immediately and the amount should not be big.

 

For Flink-13184, we want to reduce the excess container further on through avoid the blocking operation in 

ResourceManager main thread. Since the code base has changed, i will attach another PR for release-1.9.

 

I will check your log to verify whether it is expected behavior.

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191115日周五 上午6:29写道:

Yang and Till,

 

So I was able to test FLINK-12342 and it brought down the number of excess containers returned but it still doesn’t eliminate it. It’s just not as egregious as before. I uploaded a new file to the lockbox.

 

As for FLINK-13184, I tried applying the PR as it would look like if it were on top of the 1.9 code but then found something strange. Uploaded the log too. It would request the new containers log, that it received the containers and then start removing the container requests without starting execution of the job graph. Off hand, is there something else required from 1.10 for this to work as expected?

 

Thanks,

Regina

 

From: Yang Wang <[hidden email]>
Sent: Friday, November 8, 2019 6:55 AM
To: Till Rohrmann <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Thanks till.

 

I have validated the PR, and it works as expected.

Future optimization will be discussed and done at FLINK-13184.

 

Best,

Yang

 

Till Rohrmann <[hidden email]> 2019116日周三 下午8:20写道:

Just a quick update. I've pushed a fixup which corrects a bug in the PR.

 

Cheers,

Till

 

On Tue, Nov 5, 2019 at 5:09 PM Chan, Regina <[hidden email]> wrote:

Thanks for the patch. I’ll try it out and update back.

 

From: Till Rohrmann <[hidden email]>
Sent: Tuesday, November 5, 2019 9:42 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Yang Wang <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

I've taken another look at the problem I think we could improve the situation by reordering the calls we do in YarnResourceManager#onContainersAllocated. I've created a PR [1] for the re-opened issue [2]. Would it be possible for you to verify the fix? What you need to do is to check this PR out, then build Flink based on this PR and then running the test. You can check out the PR via `git fetch https://github.com/apache/flink.git pull/10089/head:FLINK-12342`. This command will fetch the PR and make it available under FLINK-12342 which you can check out and then run `mvn clean install -DskipTests` in order to build Flink. If it is easier to check out the branch from my repository, then you can find it here [3]. Thanks a lot for your help!

 

 

Cheers,

Till

 

On Fri, Nov 1, 2019 at 9:33 AM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

at the moment the community works towards the 1.10 release with a lot of features trying to be completed. The intended feature freeze is end of November. Due to this it is quite hard to tell when exactly this problem will be properly fixed but we'll try our best.

 

Cheers,

Till

 

On Thu, Oct 31, 2019 at 4:59 PM Chan, Regina <[hidden email]> wrote:

Yeah I saw FLINK-13184 earlier and started watching it. I can see the second optimization being helpful too in a large cluster. I’ll be watching this as well. Do you have an estimate as to turn around time? Would be helpful planning-wise.

 

 

From: Yang Wang <[hidden email]>
Sent: Thursday, October 31, 2019 4:08 AM
To: Chan, Regina [Engineering] <[hidden email]>
Cc: Till Rohrmann <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

I think till's analysis is right. I just want to share more information.

 

After dive into the logs of Flink resource manager and Yarn resource manager, i found that the excess

containers come from two sides.

 

** Yarn Container Allocation Mechanism **

Receive containers more than allocated is inescapable. Imagine that we want to allocate 120 containers

from Yarn. The size of container request in the heartbeat1 will be 120. When Yarn RM received the

request and can not allocate any container because of not enough resource. So the allocated containers in 

response of heartbeat1 will be 0. The Flink resource manager does not get any containers and will

set the size of container request in heartbeat2 to 120. However, Yarn resource manager has allocated

120 containers between heartbeat1 to heartbeat2. When Yarn Resource Manager receives heartbeat2, it will

set the 120 containers to response of heartbeat2. And it start to allocate for the new request of 120. Since

Flink resource manager has received all containers, it will set the size of container request in heartbeat3 to 0.

Yarn Resource Manager allocate 100 containers between heartbeat2 to heartbeat3, it will set the 100 containers

to response of heartbeat3. So Flink Resource Manager gets the 100 excess containers.

 

Note: Heartbeat means the heartbeat between Flink resource manager(Yarn client) and Yarn resource manager.

 

 

** Flink resource manager allocates more than it really needs **

Now in the onContainersAllocated of FlinkYarnResourceManager, we iterate through each container.

And each process will take more than 50ms. The most time cost at uploading {uuid}-taskmanager-conf.yaml to hdfs

and starting container. So if the allocated containers are more than 10, FlinkYarnResourceManager could not remove

container request timely and will allocate more than it really needs.

 

 

The first cause of Yarn, we could not do anything more from Flink. However, for the second, we could reduce the time

costof each allocated container so that FlinkYarnResource will allocate as it really need.  We could have two optimizations

here. The first is use NMClientAsync instead of NMClient to reduce the start container time.[1] The

second is do not upload {uuid}-taskmanager-conf.yaml, use java options or environments instead. [2]

 

 

 

 

 

Chan, Regina <[hidden email]> 20191031日周四 上午5:09写道:

Just to provide a little bit of context, this behavior is highly problematic since we run these jobs at scale. This one job when running on 1.6 over allocated 2500 containers. On 1.9, with a one-minute heartbeat interval, we were able to bring that number of excess containers down to 230. My fear is that 230 excess containers is due to the fact that we also moved this to a smaller cluster so that it doesn’t have the potential of causing wider impact it did on the main cluster. We have over 70K jobs running in a day so imagine how bad this could become so I definitely appreciate your attention to this.

 

I’m open to a minimum and max number of TaskExecutors, the max number is probably the biggest concern. Can help test this whenever it’s ready and again greatly appreciate it.

 

Separately I think this loosely ties to into another thread on the dlist so far which is the usecase of per program or per job semantics. As we have one program representing a datastore’s batch with one or many jobs per dataset representing the stages of processing. Using this paradigm I think of the program having a pool of resources to be used by the job(s) with some expiry. The pool can be enlarged through additional requests when needed with the jobs acquiring and releasing back to the pool without having to send new requests to YARN.

 

I believe perfect usage of this pool behavior would bring down the total requests from the 540 to 120 as 120+230=350 which means there’s still an additional request of 190 containers.

 

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 30, 2019 2:01 PM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina, sorry for not getting back to you earlier. I've gone through the logs and I couldn't find something suspicious. What I can see though is the following:

 

When you start the cluster, you submit a couple of jobs. This starts at 9:20. In total 120 slots are being required to run these jobs. Since you start a TaskExecutor with a single slot, you need 120 containers to run all jobs. Flink has sent all container requests by 9:21:40. So far so good. 

 

Shortly after, the cluster receives the first allocated containers. However, it lasts until 9:29:58 that Flink has received all 120 containers. I assume it is because the Hadoop cluster is quite contested. Afterwards one sees that 230 excess containers are being returned.

 

Given that the cluster is configured with yarn.heartbeat.container-request-interval: 60000, the Hadoop RM heartbeat is set to one minute. Hence, we report every minute the current number of required containers. Due to Yarn adding these requests up [1], it does not seem surprising that we end up with 230 excess containers. Assuming a constant container fulfillment we end up with 120 + 108 + 96 + ... + 12 + 0 = 540 requested containers on the Yarn RM side. This roughly matches the 120 + 230 (excess containers).

 

I'm not exactly sure how we can solve the Yarn problem. Flink would have to make sure that every container request is only sent once to the Yarn RM (e.g. canceling the container request after one heartbeat interval has been passed). However, this is not super trivial and might end up being super brittle.

 

Another idea could be to add a minimum and maximum number of `TaskExecutors` the cluster should keep alive. That way one would only pay the price of too many excess containers at startup but then the system would keep at least minimum number of TaskExecutors alive. If you wish this would imitate a bit the legacy mode Yarn behaviour where you start the cluster with a fixed number of TaskExecutors.

 

 

Cheers,

Till

 

On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

If it is a bug, i think it is critical. Could you share the job manager logs with me too? I have some time to 

analyze and hope to find the root cause.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191030日周三 上午10:55写道:

Till, were you able find anything? Do you need more logs?

 

 

From: Till Rohrmann <[hidden email]>
Sent: Saturday, October 26, 2019 1:17 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Forget my last email. I received the on time code and could access the logs.

 

Cheers,

Till

 

On Sat, Oct 26, 2019 at 6:49 PM Till Rohrmann <[hidden email]> wrote:

Hi Regina,

 

I couldn't access the log files because LockBox asked to create a new password and now it asks me for the one time code to confirm this change. It says that it will send the one time code to my registered email which I don't have.

 

Cheers,

Till

 

On Fri, Oct 25, 2019 at 10:14 PM Till Rohrmann <[hidden email]> wrote:

Great, thanks a lot Regina. I'll check the logs tomorrow. If info level is not enough, then I'll let you know.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 21:20 Chan, Regina <[hidden email]> wrote:

Till, I added you to this lockbox area where you should be able to download the logs. You should have also received an email with an account created in lockbox where you can set a password. Let me know if you have any issues.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Friday, October 25, 2019 1:24 PM
To: Chan, Regina [Engineering] <
[hidden email]>
Cc: Yang Wang <
[hidden email]>; user <[hidden email]>
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Could you provide me with the full logs of the cluster entrypoint/JobManager. I'd like to see what's going on there.

 

Cheers,

Till

 

On Fri, Oct 25, 2019, 19:10 Chan, Regina <[hidden email]> wrote:

Till,

 

We’re still seeing a large number of returned containers even with this heart beat set to something higher. Do you have hints as to what’s going on? It seems to be bursty in nature. The bursty requests cause the job to fail with the cluster not having enough resources because it’s in the process of releasing them.

“org.apache.flink.runtime.jobmanager.scheduler.NoResourceAvailableException: Could not allocate enough slots to run the job. Please make sure that the cluster has enough resources.” It causes the job to run very inconsistently.

 

Since legacy mode is now gone in 1.9, we don’t really see many options here.

 

Run Profile

Number of returned excess containers

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=500

685

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=5000

552

12G per TM, 2 slots
yarn.heartbeat.container-request-interval=10000

331

10G per TM, 1 slots
yarn.heartbeat.container-request-interval=60000

478

 

2019-10-25 09:55:51,452 INFO  org.apache.flink.runtime.executiongraph.ExecutionGraph        - Deploying CHAIN DataSource (synonym | Read Staging From File System | AVRO) -> Map (Map at readAvroFileWithFilter(FlinkReadUtils.java:78)) -> Map (Key Extractor) (14/90) (attempt #0) to container_e22_1571837093169_78279_01_000852 @ d50503-004-e22.dc.gs.com (dataPort=33579)

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000909 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000909.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000910 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000910.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000911 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000911.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000912 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000912.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000913 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000913.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000914 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000914.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000915 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000915.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000916 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000916.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000917 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000917.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000918 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000918.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000919 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000919.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000920 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000920.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000921 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000921.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000922 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000922.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000923 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000923.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000924 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000924.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000925 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000925.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000926 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000926.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000927 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000927.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000928 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000928.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000929 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000929.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000930 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000930.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000931 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000931.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000932 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000932.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000933 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000933.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000934 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000934.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000935 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000935.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000936 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000936.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000937 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000937.

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000939 - Remaining pending container requests: 0

2019-10-25 09:55:51,513 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000939.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000940 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000940.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000941 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000941.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000942 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000942.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000943 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000943.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000944 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000944.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000945 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000945.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000946 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Returning excess container container_e22_1571837093169_78279_01_000946.

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     - Received new container: container_e22_1571837093169_78279_01_000947 - Remaining pending container requests: 0

2019-10-25 09:55:51,514 INFO  org.apache.flink.yarn.YarnResourceManager                     -

 

 

From: Chan, Regina [Engineering]
Sent: Wednesday, October 23, 2019 4:51 PM
To: 'Till Rohrmann' <[hidden email]>; Yang Wang <[hidden email]>
Cc: [hidden email]
Subject: RE: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Yeah thanks for the responses. We’re in the process of testing 1.9.1 after we found https://issues.apache.org/jira/browse/FLINK-12342 as the cause of the original issue. FLINK-9455 makes sense as to why it didn’t work on legacy mode.

 

 

 

From: Till Rohrmann <[hidden email]>
Sent: Wednesday, October 23, 2019 5:32 AM
To: Yang Wang <[hidden email]>
Cc: Chan, Regina [Engineering] <[hidden email]>; [hidden email]
Subject: Re: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged

 

Hi Regina,

 

When using the FLIP-6 mode, you can control how long it takes for an idle TaskManager to be released via resourcemanager.taskmanager-timeout. Per default it is set to 30s.

 

In the Flink version you are using, 1.6.4, we do not support TaskManagers with multiple slots properly [1]. The consequence is that Flink will request too many containers if you are using FLIP-6 and configured your TaskManagers to be started with more than a single slot. With Flink >= 1.7.0 this issue has been fixed.

 

For the problem with the legacy mode it seems that there is a bug in the YarnFlinkResourceManager where we decrement the number of pending container requests by 2 instead of 1 every time a container is allocated [2]. This could explain the difference.

 

Since the Flink community no longer actively maintains Flink 1.6, I was wondering whether it would be possible for you to upgrade to a later version of Flink? I believe that your observed problems are fixed in a more recent version (1.9.1).

 

 

On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <[hidden email]> wrote:

Hi Chan,

 

After FLIP-6, the Flink ResourceManager dynamically allocate resource from Yarn on demand.

What's your flink version? On the current code base, if the pending containers in resource manager

is zero, then it will releaseall the excess containers. Could you please check the 

"Remaining pending container requests" in your jm logs?

 

On the other hand, Flink should not allocate such many resources. Do you set the `taskmanager.numberOfTaskSlots`?

The default value is 1 and will allocate containers based on your max parallelism.

 

 

Best,

Yang

 

Chan, Regina <[hidden email]> 20191023日周三 上午12:40写道:

Hi,

 

One of our Flink jobs has a lot of tiny Flink Jobs (and some larger jobs) associated with it that then request and release resources as need as per the FLIP-6 mode. Internally we track how much parallelism we’ve used before submitting the new job so that we’re bounded by the expected top cap. What we found was that the job intermittently holds onto 20-40x what is expected and thereby eating into our cluster’s overall resources. It seems as if Flink isn’t releasing the resources back to Yarn quickly enough for these.

 

As an immediate stop gap, what I tried doing was just to revert to using legacy mode hoping that the resource utilization is then at least constant as per the number of task managers + slots + memory allocated. However, we then ran into this issue. Why would the client’s pending container requests still be 60 when Yarn shows it’s been allocated? What can we do here?

 

org.apache.flink.runtime.akka.StoppingSupervisorWithoutLoggingActorKilledExceptionStrategy  - Actor failed with exception. Stopping it now.

java.lang.IllegalStateException: The RMClient's and YarnResourceManagers internal state about the number of pending container requests has diverged. Number client's pending container requests 60 != Number RM's pending container requests 0.

            at org.apache.flink.util.Preconditions.checkState(Preconditions.java:217)

            at org.apache.flink.yarn.YarnFlinkResourceManager.getPendingRequests(YarnFlinkResourceManager.java:520)

            at org.apache.flink.yarn.YarnFlinkResourceManager.containersAllocated(YarnFlinkResourceManager.java:449)

            at org.apache.flink.yarn.YarnFlinkResourceManager.handleMessage(YarnFlinkResourceManager.java:227)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.handleLeaderSessionID(FlinkUntypedActor.java:104)

            at org.apache.flink.runtime.akka.FlinkUntypedActor.onReceive(FlinkUntypedActor.java:71)

            at akka.actor.UntypedActor$$anonfun$receive$1.applyOrElse(UntypedActor.scala:165)

            at akka.actor.Actor$class.aroundReceive(Actor.scala:502)

            at akka.actor.UntypedActor.aroundReceive(UntypedActor.scala:95)

            at akka.actor.ActorCell.receiveMessage(ActorCell.scala:526)

            at akka.actor.ActorCell.invoke(ActorCell.scala:495)

            at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:257)

            at akka.dispatch.Mailbox.run(Mailbox.scala:224)

            at akka.dispatch.Mailbox.exec(Mailbox.scala:234)

            at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)

            at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)

            at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)

            at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)

 

JobManager logs: (full logs also attached)

 

2019-10-22 11:36:52,733 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000002 - Remaining pending container requests: 118

2019-10-22 11:36:52,734 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612734: Container: [ContainerId: container_e102_1569128826219_23941567_01_000002, NodeId: d49111-041.dc.gs.com:45454, NodeHttpAddress: d49111-041.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.83.235:45454 }, ] on host d49111-041.dc.gs.com

2019-10-22 11:36:52,736 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-041.dc.gs.com:45454

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000003 - Remaining pending container requests: 116

2019-10-22 11:36:52,784 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758612784: Container: [ContainerId: container_e102_1569128826219_23941567_01_000003, NodeId: d49111-162.dc.gs.com:45454, NodeHttpAddress: d49111-162.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.72.254:45454 }, ] on host d49111-162.dc.gs.com

….

Received new container: container_e102_1569128826219_23941567_01_000066 - Remaining pending container requests: 2

2019-10-22 11:36:53,409 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613409: Container: [ContainerId: container_e102_1569128826219_23941567_01_000066, NodeId: d49111-275.dc.gs.com:45454, NodeHttpAddress: d49111-275.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.50.199.239:45454 }, ] on host d49111-275.dc.gs.com

2019-10-22 11:36:53,411 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-275.dc.gs.com:45454

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000067 - Remaining pending container requests: 0

2019-10-22 11:36:53,418 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613418: Container: [ContainerId: container_e102_1569128826219_23941567_01_000067, NodeId: d49111-409.dc.gs.com:45454, NodeHttpAddress: d49111-409.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.40.203:45454 }, ] on host d49111-409.dc.gs.com

2019-10-22 11:36:53,420 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-409.dc.gs.com:45454

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000070 - Remaining pending container requests: 0

2019-10-22 11:36:53,430 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613430: Container: [ContainerId: container_e102_1569128826219_23941567_01_000070, NodeId: d49111-167.dc.gs.com:45454, NodeHttpAddress: d49111-167.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.138.251:45454 }, ] on host d49111-167.dc.gs.com

2019-10-22 11:36:53,432 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-167.dc.gs.com:45454

2019-10-22 11:36:53,439 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000072 - Remaining pending container requests: 0

2019-10-22 11:36:53,440 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613439: Container: [ContainerId: container_e102_1569128826219_23941567_01_000072, NodeId: d49111-436.dc.gs.com:45454, NodeHttpAddress: d49111-436.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.59.235.176:45454 }, ] on host d49111-436.dc.gs.com

2019-10-22 11:36:53,441 INFO  org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy  - Opening proxy : d49111-436.dc.gs.com:45454

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Received new container: container_e102_1569128826219_23941567_01_000073 - Remaining pending container requests: 0

2019-10-22 11:36:53,449 INFO  org.apache.flink.yarn.YarnFlinkResourceManager                - Launching TaskManager in container ContainerInLaunch @ 1571758613449: Container: [ContainerId: container_e102_1569128826219_23941567_01_000073, NodeId: d49111-387.dc.gs.com:45454, NodeHttpAddress: d49111-387.dc.gs.com:8042, Resource: <memory:12288, vCores:2>, Priority: 0, Token: Token { kind: ContainerToken, service: 10.51.136.247:45454 }, ] on host d49111-387.dc.gs.com

…..

 

 

Thanks,

Regina

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices

 



Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices




Your Personal Data: We may collect and process information about you that may be subject to data protection laws. For more information about how we use and disclose your personal data, how we protect your information, our legal basis to use your information, your rights and who you can contact, please refer to: www.gs.com/privacy-notices
12