mirror of https://github.com/apache/druid.git
Fix Peon not fail gracefully (#14880)
* fix Peon not fail gracefully * move methods to Task interface * fix checkstyle * extract to interface * check runThread nullability * fix merge conflict * minor refine * minor refine * fix unit test * increase latch waiting time
This commit is contained in:
parent
2f1bcd6717
commit
86087cee0a
|
@ -55,6 +55,8 @@ import java.nio.file.Paths;
|
|||
import java.util.HashMap;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.CountDownLatch;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
public abstract class AbstractTask implements Task
|
||||
{
|
||||
|
@ -101,6 +103,8 @@ public abstract class AbstractTask implements Task
|
|||
|
||||
private final ServiceMetricEvent.Builder metricBuilder = new ServiceMetricEvent.Builder();
|
||||
|
||||
private volatile CountDownLatch cleanupCompletionLatch;
|
||||
|
||||
protected AbstractTask(String id, String dataSource, Map<String, Object> context, IngestionMode ingestionMode)
|
||||
{
|
||||
this(id, null, null, dataSource, context, ingestionMode);
|
||||
|
@ -166,6 +170,7 @@ public abstract class AbstractTask implements Task
|
|||
{
|
||||
TaskStatus taskStatus = TaskStatus.running(getId());
|
||||
try {
|
||||
cleanupCompletionLatch = new CountDownLatch(1);
|
||||
String errorMessage = setup(taskToolbox);
|
||||
if (org.apache.commons.lang3.StringUtils.isNotBlank(errorMessage)) {
|
||||
return TaskStatus.failure(getId(), errorMessage);
|
||||
|
@ -178,14 +183,23 @@ public abstract class AbstractTask implements Task
|
|||
throw e;
|
||||
}
|
||||
finally {
|
||||
cleanUp(taskToolbox, taskStatus);
|
||||
try {
|
||||
cleanUp(taskToolbox, taskStatus);
|
||||
}
|
||||
finally {
|
||||
cleanupCompletionLatch.countDown();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
public abstract TaskStatus runTask(TaskToolbox taskToolbox) throws Exception;
|
||||
|
||||
@Override
|
||||
public void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception
|
||||
{
|
||||
// clear any interrupted status to ensure subsequent cleanup proceeds without interruption.
|
||||
Thread.interrupted();
|
||||
|
||||
if (!toolbox.getConfig().isEncapsulatedTask()) {
|
||||
log.debug("Not pushing task logs and reports from task.");
|
||||
return;
|
||||
|
@ -216,6 +230,24 @@ public abstract class AbstractTask implements Task
|
|||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean waitForCleanupToFinish()
|
||||
{
|
||||
try {
|
||||
if (cleanupCompletionLatch != null) {
|
||||
// block until the cleanup process completes
|
||||
return cleanupCompletionLatch.await(300, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
return true;
|
||||
}
|
||||
catch (InterruptedException e) {
|
||||
log.warn("Interrupted while waiting for task cleanUp to finish!");
|
||||
Thread.currentThread().interrupt();
|
||||
return false;
|
||||
}
|
||||
}
|
||||
|
||||
public static String getOrMakeId(@Nullable String id, final String typeName, String dataSource)
|
||||
{
|
||||
return getOrMakeId(id, typeName, dataSource, null);
|
||||
|
|
|
@ -257,6 +257,32 @@ public interface Task
|
|||
*/
|
||||
TaskStatus run(TaskToolbox toolbox) throws Exception;
|
||||
|
||||
/**
|
||||
* Performs cleanup operations after the task execution.
|
||||
* This method is intended to be overridden by tasks that need to perform
|
||||
* specific cleanup actions upon task completion or termination.
|
||||
*
|
||||
* @param toolbox Toolbox for this task
|
||||
* @param taskStatus Provides the final status of the task, indicating if the task
|
||||
* was successful, failed, or was killed.
|
||||
* @throws Exception If any error occurs during the cleanup process.
|
||||
*/
|
||||
default void cleanUp(TaskToolbox toolbox, TaskStatus taskStatus) throws Exception
|
||||
{
|
||||
}
|
||||
|
||||
/**
|
||||
* Waits for the cleanup operations to finish.
|
||||
* This method can be overridden by tasks that need to ensure that certain cleanup
|
||||
* operations have completed before proceeding further.
|
||||
*
|
||||
* @return true if the cleanup completed successfully, false otherwise.
|
||||
*/
|
||||
default boolean waitForCleanupToFinish()
|
||||
{
|
||||
return true;
|
||||
}
|
||||
|
||||
default Map<String, Object> addToContext(String key, Object val)
|
||||
{
|
||||
getContext().put(key, val);
|
||||
|
|
|
@ -185,6 +185,7 @@ public class SingleTaskBackgroundRunner implements TaskRunner, QuerySegmentWalke
|
|||
// stopGracefully for resource cleaning
|
||||
log.info("Starting graceful shutdown of task[%s].", task.getId());
|
||||
task.stopGracefully(taskConfig);
|
||||
task.waitForCleanupToFinish();
|
||||
|
||||
if (taskConfig.isRestoreTasksOnRestart() && task.canRestore()) {
|
||||
try {
|
||||
|
|
|
@ -1422,7 +1422,10 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||
{
|
||||
log.info("Stopping forcefully (status: [%s])", status);
|
||||
stopRequested.set(true);
|
||||
runThread.interrupt();
|
||||
// Interrupt if the task has started to run
|
||||
if (runThread != null) {
|
||||
runThread.interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
public void stopGracefully()
|
||||
|
|
|
@ -84,6 +84,8 @@ public class TestTasks
|
|||
@JsonTypeName("unending")
|
||||
public static class UnendingTask extends AbstractTask
|
||||
{
|
||||
private Thread runningThread;
|
||||
|
||||
@JsonCreator
|
||||
public UnendingTask(@JsonProperty("id") String id)
|
||||
{
|
||||
|
@ -105,12 +107,16 @@ public class TestTasks
|
|||
@Override
|
||||
public void stopGracefully(TaskConfig taskConfig)
|
||||
{
|
||||
if (runningThread != null) {
|
||||
runningThread.interrupt();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public TaskStatus runTask(TaskToolbox toolbox) throws Exception
|
||||
{
|
||||
while (!Thread.currentThread().isInterrupted()) {
|
||||
runningThread = Thread.currentThread();
|
||||
while (!runningThread.isInterrupted()) {
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
return TaskStatus.failure(getId(), "Dummy task status failure for testing");
|
||||
|
|
|
@ -179,14 +179,24 @@ public class SingleTaskBackgroundRunnerTest
|
|||
@Test
|
||||
public void testStop() throws ExecutionException, InterruptedException, TimeoutException
|
||||
{
|
||||
AtomicReference<Boolean> methodCallHolder = new AtomicReference<>();
|
||||
final ListenableFuture<TaskStatus> future = runner.run(
|
||||
new NoopTask(null, null, null, Long.MAX_VALUE, 0, null) // infinite task
|
||||
{
|
||||
@Override
|
||||
public boolean waitForCleanupToFinish()
|
||||
{
|
||||
methodCallHolder.set(true);
|
||||
return true;
|
||||
}
|
||||
}
|
||||
);
|
||||
runner.stop();
|
||||
Assert.assertEquals(
|
||||
TaskState.FAILED,
|
||||
future.get(1000, TimeUnit.MILLISECONDS).getStatusCode()
|
||||
);
|
||||
Assert.assertTrue(methodCallHolder.get());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -56,6 +56,7 @@ import org.apache.druid.guice.JsonConfigProvider;
|
|||
import org.apache.druid.guice.LazySingleton;
|
||||
import org.apache.druid.guice.LifecycleModule;
|
||||
import org.apache.druid.guice.ManageLifecycle;
|
||||
import org.apache.druid.guice.ManageLifecycleServer;
|
||||
import org.apache.druid.guice.PolyBind;
|
||||
import org.apache.druid.guice.QueryRunnerFactoryModule;
|
||||
import org.apache.druid.guice.QueryableModule;
|
||||
|
@ -246,7 +247,10 @@ public class CliPeon extends GuiceRunnable
|
|||
|
||||
binder.bind(TaskRunner.class).to(SingleTaskBackgroundRunner.class);
|
||||
binder.bind(QuerySegmentWalker.class).to(SingleTaskBackgroundRunner.class);
|
||||
binder.bind(SingleTaskBackgroundRunner.class).in(ManageLifecycle.class);
|
||||
// Bind to ManageLifecycleServer to ensure SingleTaskBackgroundRunner is closed before
|
||||
// its dependent services, such as DiscoveryServiceLocator and OverlordClient.
|
||||
// This order ensures that tasks can finalize their cleanup operations before service location closure.
|
||||
binder.bind(SingleTaskBackgroundRunner.class).in(ManageLifecycleServer.class);
|
||||
|
||||
bindRealtimeCache(binder);
|
||||
bindCoordinatorHandoffNotifer(binder);
|
||||
|
|
Loading…
Reference in New Issue