-
Notifications
You must be signed in to change notification settings - Fork 9.2k
YARN-11112 Avoid renewing delegation token when app is first submitte… #4198
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
d9c40fc
d753fef
610ac82
c22cbe8
ece37d6
8039901
edc1db8
2687aff
fba0d3b
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -43,6 +43,7 @@ | |
| import java.util.concurrent.TimeUnit; | ||
| import java.util.concurrent.TimeoutException; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicLong; | ||
| import java.util.concurrent.locks.ReadWriteLock; | ||
| import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
|
|
||
|
|
@@ -123,6 +124,7 @@ public class DelegationTokenRenewer extends AbstractService { | |
| private long tokenRenewerThreadTimeout; | ||
| private long tokenRenewerThreadRetryInterval; | ||
| private int tokenRenewerThreadRetryMaxAttempts; | ||
| private long clockSkewExtraTime; | ||
| private final Map<DelegationTokenRenewerEvent, Future<?>> futures = | ||
| new ConcurrentHashMap<>(); | ||
| private boolean delegationTokenRenewerPoolTrackerFlag = true; | ||
|
|
@@ -165,6 +167,8 @@ protected void serviceInit(Configuration conf) throws Exception { | |
| tokenRenewerThreadRetryMaxAttempts = | ||
| conf.getInt(YarnConfiguration.RM_DT_RENEWER_THREAD_RETRY_MAX_ATTEMPTS, | ||
| YarnConfiguration.DEFAULT_RM_DT_RENEWER_THREAD_RETRY_MAX_ATTEMPTS); | ||
| clockSkewExtraTime = conf.getLong(YarnConfiguration.RM_DT_RENEWER_THREAD_CLOCK_SKEW_TIME, | ||
| YarnConfiguration.DEFAULT_RM_DT_RENEWER_THREAD_CLOCK_SKEW_TIME); | ||
| setLocalSecretManagerAndServiceAddr(); | ||
| renewerService = createNewThreadPoolService(conf); | ||
| pendingEventQueue = new LinkedBlockingQueue<DelegationTokenRenewerEvent>(); | ||
|
|
@@ -491,12 +495,13 @@ private void handleAppSubmitEvent(AbstractDelegationTokenRenewerAppEvent evt) | |
| Set<DelegationTokenToRenew> tokenList = new HashSet<DelegationTokenToRenew>(); | ||
| boolean hasHdfsToken = false; | ||
| for (Token<?> token : tokens) { | ||
| AtomicLong tokenExpiredTime = new AtomicLong(now); | ||
| if (token.isManaged()) { | ||
| if (token.getKind().equals(HDFS_DELEGATION_KIND)) { | ||
| LOG.info(applicationId + " found existing hdfs token " + token); | ||
| hasHdfsToken = true; | ||
| } | ||
| if (skipTokenRenewal(token)) { | ||
| if (skipTokenRenewalAndUpdateExpiredTime(token, tokenExpiredTime)) { | ||
| continue; | ||
| } | ||
|
|
||
|
|
@@ -523,9 +528,14 @@ private void handleAppSubmitEvent(AbstractDelegationTokenRenewerAppEvent evt) | |
| tokenConf = getConfig(); | ||
| } | ||
| dttr = new DelegationTokenToRenew(Arrays.asList(applicationId), token, | ||
| tokenConf, now, shouldCancelAtEnd, evt.getUser()); | ||
| tokenConf, tokenExpiredTime.get(), shouldCancelAtEnd, evt.getUser()); | ||
|
|
||
| try { | ||
| renewToken(dttr); | ||
| // if expire date is not greater than now, renew token. | ||
| // add extra time in case of clock skew | ||
| if (tokenExpiredTime.get() <= now + clockSkewExtraTime) { | ||
| renewToken(dttr); | ||
| } | ||
| } catch (IOException ioe) { | ||
| if (ioe instanceof SecretManager.InvalidToken | ||
| && dttr.maxDate < Time.now() | ||
|
|
@@ -617,8 +627,9 @@ public boolean cancel() { | |
| * Skip renewing token if the renewer of the token is set to "" | ||
| * Caller is expected to have examined that token.isManaged() returns | ||
| * true before calling this method. | ||
| * if identifier is not null, get the max expired time from token identifier. | ||
| */ | ||
| private boolean skipTokenRenewal(Token<?> token) | ||
| private boolean skipTokenRenewalAndUpdateExpiredTime(Token<?> token, AtomicLong expiredTime) | ||
| throws IOException { | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
|
|
@@ -627,6 +638,7 @@ private boolean skipTokenRenewal(Token<?> token) | |
| if (identifier == null) { | ||
| return false; | ||
| } | ||
| expiredTime.set(identifier.getMaxDate()); | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Do we just to reset the expireTime only when the result of
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. This should be identifier.getIssueDate() |
||
| Text renewer = identifier.getRenewer(); | ||
| return (renewer != null && renewer.toString().equals("")); | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Just suggest to check if delegation token need to renew while app is just submitted. |
||
| } | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sorry I didn't get why add clock skew condition check here. I think we should file another JIRA to fix if it is issue indeed.