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 <regina.c...@gs.com> 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 <danrtsey...@gmail.com>
> *Sent:* Thursday, October 31, 2019 4:08 AM
> *To:* Chan, Regina [Engineering] <regina.c...@ny.email.gs.com>
> *Cc:* Till Rohrmann <trohrm...@apache.org>; user <user@flink.apache.org>
> *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]
>
>
>
>
>
>
>
>
>
> 1.https://issues.apache.org/jira/browse/FLINK-13184
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_FLINK-2D13184&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=yB3bER8VgYAerfYBH0iE2pz4fPo2FT5lsCvRuyeH06E&s=r9gRqjcTahgELRE0zGt1akX4QMHHpUZUjKUXAG6X5yY&e=>
>
> 2. https://issues.apache.org/jira/browse/FLINK-14582
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_FLINK-2D14582&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=yB3bER8VgYAerfYBH0iE2pz4fPo2FT5lsCvRuyeH06E&s=lu9e-E_1F1Cp2yZYrQ3KQu12eGlP5rQE7J8PMm8ktGw&e=>
>
>
>
> Chan, Regina <regina.c...@gs.com> 于2019年10月31日周四 上午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 <trohrm...@apache.org>
> *Sent:* Wednesday, October 30, 2019 2:01 PM
> *To:* Yang Wang <danrtsey...@gmail.com>
> *Cc:* Chan, Regina [Engineering] <regina.c...@ny.email.gs.com>; user <
> user@flink.apache.org>
> *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.
>
>
>
> [1] https://issues.apache.org/jira/browse/YARN-1902
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_YARN-2D1902&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=L6nG50CITACvmh48HSUhbjjzikT4SYEfQw_CnE-_qrI&s=vQK5H4etJXHuHIUCaMO8oAmm7ta4OWKDq_ri3-fLgIA&e=>
>
>
>
> Cheers,
>
> Till
>
>
>
> On Wed, Oct 30, 2019 at 4:11 AM Yang Wang <danrtsey...@gmail.com> 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 <regina.c...@gs.com> 于2019年10月30日周三 上午10:55写道:
>
> Till, were you able find anything? Do you need more logs?
>
>
>
>
>
> *From:* Till Rohrmann <trohrm...@apache.org>
> *Sent:* Saturday, October 26, 2019 1:17 PM
> *To:* Chan, Regina [Engineering] <regina.c...@ny.email.gs.com>
> *Cc:* Yang Wang <danrtsey...@gmail.com>; user <user@flink.apache.org>
> *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 <trohrm...@apache.org>
> 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 <trohrm...@apache.org>
> 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 <regina.c...@gs.com> 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 <trohrm...@apache.org>
> *Sent:* Friday, October 25, 2019 1:24 PM
> *To:* Chan, Regina [Engineering] <regina.c...@ny.email.gs.com>
> *Cc:* Yang Wang <danrtsey...@gmail.com>; user <user@flink.apache.org>
> *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 <regina.c...@gs.com> 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' <trohrm...@apache.org>; Yang Wang <
> danrtsey...@gmail.com>
> *Cc:* user@flink.apache.org
> *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
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_FLINK-2D12342&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=DJ6ltFlmNzuzhG0h4KzRpt9-12NF2eUOcHuOozTzpAk&s=mNfpHaW_AxiT2VmwVQ1kyTmfKQuSlF6yEvH6YQpDH-8&e=>
> as the cause of the original issue. FLINK-9455 makes sense as to why it
> didn’t work on legacy mode.
>
>
>
>
>
>
>
> *From:* Till Rohrmann <trohrm...@apache.org>
> *Sent:* Wednesday, October 23, 2019 5:32 AM
> *To:* Yang Wang <danrtsey...@gmail.com>
> *Cc:* Chan, Regina [Engineering] <regina.c...@ny.email.gs.com>;
> user@flink.apache.org
> *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).
>
>
>
> [1] https://issues.apache.org/jira/browse/FLINK-9455
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__issues.apache.org_jira_browse_FLINK-2D9455&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=ZRpDhwXKaaezw-9M5MISk2_jl7BSQMqQHNam2urC3wo&e=>
>
> [2]
> https://github.com/apache/flink/blob/release-1.6.4/flink-yarn/src/main/java/org/apache/flink/yarn/YarnFlinkResourceManager.java#L457
> <https://urldefense.proofpoint.com/v2/url?u=https-3A__github.com_apache_flink_blob_release-2D1.6.4_flink-2Dyarn_src_main_java_org_apache_flink_yarn_YarnFlinkResourceManager.java-23L457&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=N10tDU0UCmVy2WpUN3w2yNRTzvi8Yl79ryhV0icpILE&e=>
>
>
>
> Cheers,
>
> Till
>
>
>
> On Wed, Oct 23, 2019 at 10:37 AM Yang Wang <danrtsey...@gmail.com> 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 <regina.c...@gs.com> 于2019年10月23日周三 上午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
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.59.83.235-3A45454&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=2dLvERgldwstlXieBLCEFT5CaNQmaiQuZ_RyceDe52s&e=>
> }, ] 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
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.59.72.254-3A45454&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=Ol_2CEUzRioQFPGNmHf4fQbCwPwQ24HIRP3SejFYQnY&e=>
> }, ] 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
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.50.199.239-3A45454&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=CrPBHrAgVH6EvaoXyJCdOsipDIFwk0zEouDSsgK_Ctg&e=>
> }, ] 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
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.59.40.203-3A45454&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=ztlqNS0esLyb8yX2V9ZJ3Oi5KQftlm2GDop27L0HFmQ&e=>
> }, ] 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
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.51.138.251-3A45454&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=nODw9DSkWxN9vGI51uhus2-Y4JgHzUdKDNtk9GYBBwo&e=>
> }, ] 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
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.59.235.176-3A45454&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=_JSGjbn3TMY5B3hwbH0o6ybTfAMVzVwLZeH_dCfFYAo&e=>
> }, ] 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
> <https://urldefense.proofpoint.com/v2/url?u=http-3A__10.51.136.247-3A45454&d=DwMFaQ&c=7563p3e2zaQw0AB1wrFVgyagb2IE5rTZOYPxLxfZlX4&r=vus_2CMQfE0wKmJ4Q_gOWWsBmKlgzMeEwtqShIeKvak&m=k2KD7mNjGSdTsB7265E5xpIqZXzOPAfcl6p_Fi7Do78&s=kcJvHJwB43UAGSBBCXT6i-9MOUPQt4_HpSt3EnZX7YE&e=>
> }, ] 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
>

Reply via email to