[
https://issues.apache.org/jira/browse/YARN-11178?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17709951#comment-17709951
]
ASF GitHub Bot commented on YARN-11178:
---------------------------------------
Hexiaoqiao commented on code in PR #4435:
URL: https://github.com/apache/hadoop/pull/4435#discussion_r1161241461
##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/conf/YarnConfiguration.java:
##########
@@ -819,6 +819,15 @@ public static boolean isAclEnabled(Configuration conf) {
public static final int DEFAULT_RM_DT_RENEWER_THREAD_RETRY_MAX_ATTEMPTS =
10;
+ /**
+ * The setting is used to control delegation token renewer thread perform
backoff
+ * waiting when there are no renewer event futures to avoid CPU busy idling.
+ */
+ public static final String RM_DT_RENEWER_THREAD_IDLE_BACKOFF_MS =
+ RM_PREFIX + "delegation-token-renewer.thread-idle-backoff-ms";
+ public static final long DEFAULT_RM_DT_RENEWER_THREAD_IDLE_BACKOFF_MS =
+ TimeUnit.SECONDS.toMillis(3); // 3 Seconds
Review Comment:
Just suggest to change `TimeUnit.SECONDS.toMillis(3)` to 180000 directly to
keep same dimension as Key/Value name says (MS/ms) and avoid ambiguity for end
user.
##########
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/yarn-default.xml:
##########
@@ -1148,6 +1148,16 @@
<value>60s</value>
</property>
+ <property>
+ <description>
+ The setting is used to control each RM DelegationTokenRenewer thread
perform backoff
+ waiting when there are no renewer event futures to avoid CPU busy idling.
+ the default value is 3 seconds.
+ </description>
+
<name>yarn.resourcemanager.delegation-token-renewer.thread-idle-backoff-ms</name>
+ <value>3s</value>
Review Comment:
+1 as mentioned above. We should keep the same dimension for both key and
value of configuration.
> Avoid CPU busy idling and resource wasting in
> DelegationTokenRenewerPoolTracker thread
> --------------------------------------------------------------------------------------
>
> Key: YARN-11178
> URL: https://issues.apache.org/jira/browse/YARN-11178
> Project: Hadoop YARN
> Issue Type: Bug
> Components: resourcemanager, security
> Affects Versions: 3.3.1, 3.3.2, 3.3.3, 3.3.4
> Environment: Hadoop 3.3.3 with Kerberos, Ranger 2.1.0, Hive 2.3.7 and
> Spark 3.0.3
> Reporter: Lennon Chin
> Priority: Minor
> Labels: pull-request-available
> Attachments: YARN-11178.CPU idling busy 100% before optimized.png,
> YARN-11178.CPU normal after optimized.png, YARN-11178.CPU profile for idling
> busy 100% before optimized.html, YARN-11178.CPU profile for idling busy 100%
> before optimized.png, YARN-11178.CPU profile for normal after optimized.html,
> YARN-11178.CPU profile for normal after optimized.png
>
> Time Spent: 1h 40m
> Remaining Estimate: 0h
>
> The DelegationTokenRenewerPoolTracker thread is busy wasting CPU resource in
> empty poll iterate when there is no delegation token renewer event task in
> the futures map:
> {code:java}
> //
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.DelegationTokenRenewerPoolTracker#run
> @Override
> public void run() {
> // this while true loop is busy when the `futures` is empty
> while (true) {
> for (Map.Entry<DelegationTokenRenewerEvent, Future<?>> entry : futures
> .entrySet()) {
> DelegationTokenRenewerEvent evt = entry.getKey();
> Future<?> future = entry.getValue();
> try {
> future.get(tokenRenewerThreadTimeout, TimeUnit.MILLISECONDS);
> } catch (TimeoutException e) {
> // Cancel thread and retry the same event in case of timeout
> if (future != null && !future.isDone() && !future.isCancelled()) {
> future.cancel(true);
> futures.remove(evt);
> if (evt.getAttempt() < tokenRenewerThreadRetryMaxAttempts) {
> renewalTimer.schedule(
> getTimerTask((AbstractDelegationTokenRenewerAppEvent) evt),
> tokenRenewerThreadRetryInterval);
> } else {
> LOG.info(
> "Exhausted max retry attempts {} in token renewer "
> + "thread for {}",
> tokenRenewerThreadRetryMaxAttempts, evt.getApplicationId());
> }
> }
> } catch (Exception e) {
> LOG.info("Problem in submitting renew tasks in token renewer "
> + "thread.", e);
> }
> }
> }
> }{code}
> A better way to avoid CPU idling is waiting for some time when the `futures`
> map is empty, and when the renewer task done or cancelled, we should remove
> the task future in `futures` map to avoid memory leak:
> {code:java}
> @Override
> public void run() {
> while (true) {
> // waiting for some time when futures map is empty
> if (futures.isEmpty()) {
> synchronized (this) {
> try {
> // waiting for tokenRenewerThreadTimeout milliseconds
> long waitingTimeMs = Math.min(10000, Math.max(500,
> tokenRenewerThreadTimeout));
> LOG.info("Delegation token renewer pool is empty, waiting for {}
> ms.", waitingTimeMs);
> wait(waitingTimeMs);
> } catch (InterruptedException e) {
> LOG.warn("Delegation token renewer pool tracker waiting interrupt
> occurred.");
> Thread.currentThread().interrupt();
> }
> }
> if (futures.isEmpty()) {
> continue;
> }
> }
> for (Map.Entry<DelegationTokenRenewerEvent, Future<?>> entry : futures
> .entrySet()) {
> DelegationTokenRenewerEvent evt = entry.getKey();
> Future<?> future = entry.getValue();
> try {
> future.get(tokenRenewerThreadTimeout, TimeUnit.MILLISECONDS);
> } catch (TimeoutException e) {
> // Cancel thread and retry the same event in case of timeout
> if (future != null && !future.isDone() && !future.isCancelled()) {
> future.cancel(true);
> futures.remove(evt);
> if (evt.getAttempt() < tokenRenewerThreadRetryMaxAttempts) {
> renewalTimer.schedule(
> getTimerTask((AbstractDelegationTokenRenewerAppEvent) evt),
> tokenRenewerThreadRetryInterval);
> } else {
> LOG.info(
> "Exhausted max retry attempts {} in token renewer "
> + "thread for {}",
> tokenRenewerThreadRetryMaxAttempts, evt.getApplicationId());
> }
> }
> } catch (Exception e) {
> LOG.info("Problem in submitting renew tasks in token renewer "
> + "thread.", e);
> }
> // remove done and cancelled task
> if (future.isDone() || future.isCancelled()) {
> try {
> futures.remove(evt);
> LOG.info("Removed done or cancelled renew tasks of {} in token
> renewer thread.", evt.getApplicationId());
> } catch (Exception e) {
> LOG.warn("Problem in removing done or cancelled renew tasks in
> token renewer thread.", e);
> }
> }
> }
> }
> } {code}
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]