[
https://issues.apache.org/jira/browse/YARN-11719?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17885553#comment-17885553
]
ASF GitHub Bot commented on YARN-11719:
---------------------------------------
zeekling opened a new pull request, #7077:
URL: https://github.com/apache/hadoop/pull/7077
<!--
Thanks for sending a pull request!
1. If this is your first time, please read our contributor guidelines:
https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
2. Make sure your PR title starts with JIRA issue id, e.g.,
'HADOOP-17799. Your PR title ...'.
-->
### Description of PR
PR for https://issues.apache.org/jira/browse/YARN-11719
### How was this patch tested?
### For code changes:
- [ ] Does the title or this PR starts with the corresponding JIRA issue id
(e.g. 'HADOOP-17799. Your PR title ...')?
- [ ] Object storage: have the integration tests been executed and the
endpoint declared according to the connector-specific documentation?
- [ ] If adding new dependencies to the code, are these dependencies
licensed in a way that is compatible for inclusion under [ASF
2.0](http://www.apache.org/legal/resolved.html#category-a)?
- [ ] If applicable, have you updated the `LICENSE`, `LICENSE-binary`,
`NOTICE-binary` files?
> The job is stuck in the new state.
> ----------------------------------
>
> Key: YARN-11719
> URL: https://issues.apache.org/jira/browse/YARN-11719
> Project: Hadoop YARN
> Issue Type: Bug
> Affects Versions: 3.3.1
> Reporter: zeekling
> Priority: Major
>
> After I restarted the router in the production environment, several jobs
> remained in the new state. and i found related log here.
>
> {code:java}
> 2024-08-30 00:12:41,380 | WARN | DelegationTokenRenewer #667 | Unable to add
> the application to the delegation token renewer. |
> DelegationTokenRenewer.java:1215
> java.io.IOException: Failed to renew token: Kind: HDFS_DELEGATION_TOKEN,
> Service: ha-hdfs:nsfed, Ident: (token for admintest: HDFS_DELEGATION_TOKEN
> owner=admintest@9FCE074E_691F_480F_98F5_58C1CA310829.COM, renewer=mapred,
> realUser=, issueDate=1724947875776, maxDate=1725552675776,
> sequenceNumber=156, masterKeyId=116)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:641)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.access$2200(DelegationTokenRenewer.java:86)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.handleDTRenewerAppSubmitEvent(DelegationTokenRenewer.java:1211)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.run(DelegationTokenRenewer.java:1188)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:750)Caused by:
> java.io.InterruptedIOException: Retry interrupted
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.processWaitTimeAndRetryInfo(RetryInvocationHandler.java:141)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:112)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:366)
> at com.sun.proxy.$Proxy96.renewDelegationToken(Unknown Source)
> at org.apache.hadoop.hdfs.DFSClient$Renewer.renew(DFSClient.java:849)
> at org.apache.hadoop.security.token.Token.renew(Token.java:498)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$1.run(DelegationTokenRenewer.java:771)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$1.run(DelegationTokenRenewer.java:768)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at
> org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1890)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.renewToken(DelegationTokenRenewer.java:767)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer.handleAppSubmitEvent(DelegationTokenRenewer.java:627)
>
> ... 8 more
> Caused by: java.lang.InterruptedException: sleep interrupted
> at java.lang.Thread.sleep(Native Method)
> at
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.processWaitTimeAndRetryInfo(RetryInvocationHandler.java:135)
> ... 20 more
> 2024-08-30 00:12:41,380 | WARN | DelegationTokenRenewer #667 |
> AsyncDispatcher thread interrupted | AsyncDispatcher.java:437
> java.lang.InterruptedException
> at
> java.util.concurrent.locks.AbstractQueuedSynchronizer.acquireInterruptibly(AbstractQueuedSynchronizer.java:1233)
> at
> java.util.concurrent.locks.ReentrantLock.lockInterruptibly(ReentrantLock.java:335)
> at
> java.util.concurrent.LinkedBlockingQueue.put(LinkedBlockingQueue.java:339)
> at
> org.apache.hadoop.yarn.event.AsyncDispatcher$GenericEventHandler.handle(AsyncDispatcher.java:434)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.handleDTRenewerAppSubmitEvent(DelegationTokenRenewer.java:1221)
> at
> org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer$DelegationTokenRenewerRunnable.run(DelegationTokenRenewer.java:1188)
> at
> java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
> at java.util.concurrent.FutureTask.run(FutureTask.java:266)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> at java.lang.Thread.run(Thread.java:750)
> 2024-08-30 00:12:41,381 | WARN | DelegationTokenRenewer #667 | Caught
> exception in thread DelegationTokenRenewer #667: | ExecutorHelper.java:63
> java.util.concurrent.CancellationException
> at java.util.concurrent.FutureTask.report(FutureTask.java:121)
> at java.util.concurrent.FutureTask.get(FutureTask.java:192)
> at
> org.apache.hadoop.util.concurrent.ExecutorHelper.logThrowableFromAfterExecute(ExecutorHelper.java:48)
> at
> org.apache.hadoop.util.concurrent.HadoopThreadPoolExecutor.afterExecute(HadoopThreadPoolExecutor.java:90)
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1157)
> {code}
>
> params:
> yarn.resourcemanager.delegation-token-renewer.thread-time=80S
> dfs.client.socket-timeout=60S
> When the Router is restarted, RM is renewing the token. At this time, the
> token renewal will try multiple times, and it will sleep for a while between
> each retry. After more than 80 seconds, the token renewal thread will be
> interrupted by the following code
>
> {code:java}
> DelegationTokenRenewerEvent evt = dtrf.getEvt();
> Future<?> future = dtrf.getFuture();
> try {
> future.get(tokenRenewerThreadTimeout, TimeUnit.MILLISECONDS);
> } catch (TimeoutException e) {
> // Cancel thread and retry the same event in case of timeout.
> if (!future.isDone() && !future.isCancelled()) {
> future.cancel(true);
> 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}
> After the interruption, it will be captured by the following code, and the
> interruption will be re-triggered, and an exception will be thrown. The renew
> token operation fails, and the state machine of the job needs to change from
> new to rejected.
> {code:java}
> try {
> Thread.sleep(retryInfo.delay);
> } catch (InterruptedException e) {
> Thread.currentThread().interrupt();
> if (LOG.isDebugEnabled()) {
> LOG.debug("Interrupted while waiting to retry", e);
> }
> InterruptedIOException intIOE = new InterruptedIOException(
> "Retry interrupted");
> intIOE.initCause(e);
> throw intIOE;
> } {code}
> However, since the interrupt signal is re-triggered, the interrupt signal
> will be detected in the following code of AsyncDispatcher.java, resulting in
> the failure of state transition.
> {code:java}
> try {
> eventQueue.put(event);
> } catch (InterruptedException e) {
> if (!stopped) {
> LOG.warn("AsyncDispatcher thread interrupted", e);
> }
> // Need to reset drained flag to true if event queue is empty,
> // otherwise dispatcher will hang on stop.
> drained = eventQueue.isEmpty();
> throw new YarnRuntimeException(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]