Fix RejectExecutionHandler of Blocking Single Threaded executor (#17146)

Throw RejectedExecutionException when submitting tasks to executor that has been shut down.
This commit is contained in:
Hardik Bajaj 2024-10-15 22:02:34 +05:30 committed by GitHub
parent c2149d59a7
commit 32ce341a6c
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
2 changed files with 38 additions and 0 deletions

View File

@ -159,6 +159,9 @@ public class Execs
@Override
public void rejectedExecution(Runnable r, ThreadPoolExecutor executor)
{
if (executor.isShutdown()) {
throw new RejectedExecutionException("Executor is shutdown, rejecting task");
}
try {
executor.getQueue().put(r);
}

View File

@ -19,15 +19,20 @@
package org.apache.druid.concurrent;
import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
import org.junit.Assert;
import org.junit.Test;
import java.util.concurrent.Callable;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.atomic.AtomicInteger;
public class ExecsTest
@ -118,6 +123,36 @@ public class ExecsTest
producer.shutdown();
}
@Test
public void testTaskAddedToShutdownExecutorThrowsException() throws Exception
{
// The implementation of Execs.newBlockingSingleThreaded() rejectedExecutionHandler should not add tasks when it's in shutDown state
// When a SynchronousQueue is used in executor and a task is put in it in ShutDown state, it will forever stuck in WAITING state
// as executor will not take() the task to schedule it.
final ListeningExecutorService intermediateTempExecutor = MoreExecutors.listeningDecorator(
Execs.newBlockingSingleThreaded("[TASK_ID]-appenderator-abandon", 0)
);
Callable<Void> task = () -> {
try {
Thread.sleep(500); // Simulate long-running task
}
catch (InterruptedException e) {
Thread.currentThread().interrupt(); // Restore interrupted status
}
return null;
};
// Submit multiple tasks together
Assert.assertNotNull(intermediateTempExecutor.submit(task));
Assert.assertNotNull(intermediateTempExecutor.submit(task));
intermediateTempExecutor.shutdownNow();
// Submit task after shutDown / shutDownNow should not be added in queue
Assert.assertThrows(RejectedExecutionException.class, () -> intermediateTempExecutor.submit(task));
Assert.assertTrue(intermediateTempExecutor.awaitTermination(10, TimeUnit.SECONDS));
Assert.assertTrue(intermediateTempExecutor.isShutdown());
}
@Test
public void testDirectExecutorFactory()
{