Skip to content
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

ThreadPools: schedule a timeout check after adding command to queue #12319

Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@
import java.util.List;
import java.util.Queue;
import java.util.concurrent.*;
import java.util.concurrent.atomic.AtomicBoolean;
import java.util.concurrent.atomic.AtomicLong;

/**
Expand Down Expand Up @@ -105,6 +106,7 @@ public void execute(Runnable command, final ScheduledExecutorService timer, fina
} else if (!(command instanceof PrioritizedFutureTask)) { // it might be a callable wrapper...
command = new TieBreakingPrioritizedRunnable(command, Priority.NORMAL, insertionOrder.incrementAndGet());
}
super.execute(command);
if (timeout.nanos() >= 0) {
if (command instanceof TieBreakingPrioritizedRunnable) {
((TieBreakingPrioritizedRunnable) command).scheduleTimeout(timer, timeoutCallback, timeout);
Expand All @@ -114,7 +116,6 @@ public void execute(Runnable command, final ScheduledExecutorService timer, fina
throw new UnsupportedOperationException("Execute with timeout is not supported for future tasks");
}
}
super.execute(command);
}

@Override
Expand Down Expand Up @@ -161,7 +162,8 @@ private final class TieBreakingPrioritizedRunnable extends PrioritizedRunnable {

private Runnable runnable;
private final long insertionOrder;
private ScheduledFuture<?> timeoutFuture;
private volatile ScheduledFuture<?> timeoutFuture;
private volatile boolean started = false;

TieBreakingPrioritizedRunnable(PrioritizedRunnable runnable, long insertionOrder) {
this(runnable, runnable.priority(), insertionOrder);
Expand All @@ -175,6 +177,7 @@ private final class TieBreakingPrioritizedRunnable extends PrioritizedRunnable {

@Override
public void run() {
started = true;
FutureUtils.cancel(timeoutFuture);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would have helped me a lot to understand the logic if you added a comment here saying that timeoutFuture might not have been initialized here yet and because of this we have to try cleaning it here and in scheduleTimeout.

runAndClean(runnable);
}
Expand All @@ -197,6 +200,10 @@ public void run() {
}
}
}, timeValue.nanos(), TimeUnit.NANOSECONDS);
if (started) {
// if the actual action already it might have missed the setting of the future. Clean it ourselves.
FutureUtils.cancel(timeoutFuture);
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -253,7 +253,7 @@ public void testTimeoutCleanup() throws Exception {
public void run() {
invoked.countDown();
}
}, timer, TimeValue.timeValueMillis(1000), new Runnable() {
}, timer, TimeValue.timeValueHours(1), new Runnable() {
@Override
public void run() {
// We should never get here
Expand Down