YARN-11489. Fix memory leak of DelegationTokenRenewer futures in DelegationTokenRenewerPoolTracker. (#5629). Contributed by Chun Chen.

Signed-off-by: He Xiaoqiao <hexiaoqiao@apache.org>
This commit is contained in:
Chun Chen 2023-05-14 21:38:04 +08:00 committed by GitHub
parent 251439d769
commit 11af08d67a
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
1 changed files with 56 additions and 28 deletions

View File

@ -123,8 +123,8 @@ public class DelegationTokenRenewer extends AbstractService {
private long tokenRenewerThreadTimeout; private long tokenRenewerThreadTimeout;
private long tokenRenewerThreadRetryInterval; private long tokenRenewerThreadRetryInterval;
private int tokenRenewerThreadRetryMaxAttempts; private int tokenRenewerThreadRetryMaxAttempts;
private final Map<DelegationTokenRenewerEvent, Future<?>> futures = private final LinkedBlockingQueue<DelegationTokenRenewerFuture> futures =
new ConcurrentHashMap<>(); new LinkedBlockingQueue<>();
private boolean delegationTokenRenewerPoolTrackerFlag = true; private boolean delegationTokenRenewerPoolTrackerFlag = true;
// this config is supposedly not used by end-users. // this config is supposedly not used by end-users.
@ -227,7 +227,7 @@ public class DelegationTokenRenewer extends AbstractService {
if (isServiceStarted) { if (isServiceStarted) {
Future<?> future = Future<?> future =
renewerService.submit(new DelegationTokenRenewerRunnable(evt)); renewerService.submit(new DelegationTokenRenewerRunnable(evt));
futures.put(evt, future); futures.add(new DelegationTokenRenewerFuture(evt, future));
} else { } else {
pendingEventQueue.add(evt); pendingEventQueue.add(evt);
int qSize = pendingEventQueue.size(); int qSize = pendingEventQueue.size();
@ -998,33 +998,35 @@ public class DelegationTokenRenewer extends AbstractService {
@Override @Override
public void run() { public void run() {
while (true) { while (true) {
for (Map.Entry<DelegationTokenRenewerEvent, Future<?>> entry : futures DelegationTokenRenewerFuture dtrf;
.entrySet()) { try {
DelegationTokenRenewerEvent evt = entry.getKey(); dtrf = futures.take();
Future<?> future = entry.getValue(); } catch (InterruptedException e) {
try { LOG.debug("DelegationTokenRenewer pool tracker interrupted");
future.get(tokenRenewerThreadTimeout, TimeUnit.MILLISECONDS); return;
} catch (TimeoutException e) { }
DelegationTokenRenewerEvent evt = dtrf.getEvt();
// Cancel thread and retry the same event in case of timeout Future<?> future = dtrf.getFuture();
if (future != null && !future.isDone() && !future.isCancelled()) { try {
future.cancel(true); future.get(tokenRenewerThreadTimeout, TimeUnit.MILLISECONDS);
futures.remove(evt); } catch (TimeoutException e) {
if (evt.getAttempt() < tokenRenewerThreadRetryMaxAttempts) { // Cancel thread and retry the same event in case of timeout.
renewalTimer.schedule( if (!future.isDone() && !future.isCancelled()) {
getTimerTask((AbstractDelegationTokenRenewerAppEvent) evt), future.cancel(true);
tokenRenewerThreadRetryInterval); if (evt.getAttempt() < tokenRenewerThreadRetryMaxAttempts) {
} else { renewalTimer.schedule(
LOG.info( getTimerTask((AbstractDelegationTokenRenewerAppEvent) evt),
"Exhausted max retry attempts {} in token renewer " tokenRenewerThreadRetryInterval);
+ "thread for {}", } else {
tokenRenewerThreadRetryMaxAttempts, evt.getApplicationId()); 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);
} }
} catch (Exception e) {
LOG.info("Problem in submitting renew tasks in token renewer "
+ "thread.", e);
} }
} }
} }
@ -1192,6 +1194,32 @@ public class DelegationTokenRenewer extends AbstractService {
} }
} }
private static class DelegationTokenRenewerFuture {
private DelegationTokenRenewerEvent evt;
private Future<?> future;
DelegationTokenRenewerFuture(DelegationTokenRenewerEvent evt,
Future<?> future) {
this.future = future;
this.evt = evt;
}
public DelegationTokenRenewerEvent getEvt() {
return evt;
}
public void setEvt(DelegationTokenRenewerEvent evt) {
this.evt = evt;
}
public Future<?> getFuture() {
return future;
}
public void setFuture(Future<?> future) {
this.future = future;
}
}
// only for testing // only for testing
protected ConcurrentMap<Token<?>, DelegationTokenToRenew> getAllTokens() { protected ConcurrentMap<Token<?>, DelegationTokenToRenew> getAllTokens() {
return allTokens; return allTokens;