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:
parent
251439d769
commit
11af08d67a
@ -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 @@ private void processDelegationTokenRenewerEvent(
|
|||||||
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,18 +998,21 @@ private final class DelegationTokenRenewerPoolTracker
|
|||||||
@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) {
|
||||||
|
LOG.debug("DelegationTokenRenewer pool tracker interrupted");
|
||||||
|
return;
|
||||||
|
}
|
||||||
|
DelegationTokenRenewerEvent evt = dtrf.getEvt();
|
||||||
|
Future<?> future = dtrf.getFuture();
|
||||||
try {
|
try {
|
||||||
future.get(tokenRenewerThreadTimeout, TimeUnit.MILLISECONDS);
|
future.get(tokenRenewerThreadTimeout, TimeUnit.MILLISECONDS);
|
||||||
} catch (TimeoutException e) {
|
} catch (TimeoutException e) {
|
||||||
|
// Cancel thread and retry the same event in case of timeout.
|
||||||
// Cancel thread and retry the same event in case of timeout
|
if (!future.isDone() && !future.isCancelled()) {
|
||||||
if (future != null && !future.isDone() && !future.isCancelled()) {
|
|
||||||
future.cancel(true);
|
future.cancel(true);
|
||||||
futures.remove(evt);
|
|
||||||
if (evt.getAttempt() < tokenRenewerThreadRetryMaxAttempts) {
|
if (evt.getAttempt() < tokenRenewerThreadRetryMaxAttempts) {
|
||||||
renewalTimer.schedule(
|
renewalTimer.schedule(
|
||||||
getTimerTask((AbstractDelegationTokenRenewerAppEvent) evt),
|
getTimerTask((AbstractDelegationTokenRenewerAppEvent) evt),
|
||||||
@ -1028,7 +1031,6 @@ public void run() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* This will run as a separate thread and will process individual events. It
|
* This will run as a separate thread and will process individual events. It
|
||||||
@ -1192,6 +1194,32 @@ public void setAttempt(int attempt) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
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;
|
||||||
|
Loading…
Reference in New Issue
Block a user