KAFKA-15326: [9/N] Start and stop executors and cornercases (#14281)
* Implements start and stop of task executors
* Introduce flush operation to keep consumer operations out of the processing threads
* Fixes corner case: handle requested unassignment during shutdown
* Fixes corner case: handle race between voluntary unassignment and requested unassigment
* Fixes corner case: task locking future completes for the empty set
* Fixes corner case: we should not reassign a task with an uncaught exception to a task executor
* Improved logging
* Number of threads controlled from outside, of the TaskManager
Reviewers: Bruno Cadonna <bruno@confluent.io>
@ -41,8 +41,8 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -41,8 +41,8 @@ public class DefaultTaskExecutor implements TaskExecutor {
@ -57,26 +57,52 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -57,26 +57,52 @@ public class DefaultTaskExecutor implements TaskExecutor {
publicvoidrun(){
log.info("Task executor thread started");
try{
while(isRunning.get()){
runOnce(time.milliseconds());
while(!shutdownRequested.get()){
try{
runOnce(time.milliseconds());
}catch(finalStreamsExceptione){
handleException(e);
}catch(finalExceptione){
handleException(newStreamsException(e));
}
}
}catch(finalStreamsExceptione){
handleException(e);
}catch(finalExceptione){
handleException(newStreamsException(e));
}finally{
if(currentTask!=null){
log.debug("Releasing task {} due to shutdown.",currentTask.id());
log.debug("Releasing task {} due to uncaught exception.",currentTask.id());
unassignCurrentTask();
}else{
// If we do not currently have a task assigned and still get an error, this is fatal for the executor thread
throwe;
@ -84,11 +110,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -84,11 +110,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
@ -105,6 +127,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -105,6 +127,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
log.trace("processed a record for {}",currentTask.id());
progressed=true;
}
}
@ -121,6 +144,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -121,6 +144,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
}
if(!progressed){
log.debug("Releasing task {} because we are not making progress.",currentTask.id());
unassignCurrentTask();
}
}
@ -131,10 +155,10 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -131,10 +155,10 @@ public class DefaultTaskExecutor implements TaskExecutor {
@ -168,9 +192,10 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -168,9 +192,10 @@ public class DefaultTaskExecutor implements TaskExecutor {
if(currentTask==null)
thrownewIllegalStateException("Does not own any task while being ask to unassign from task manager");
// flush the task before giving it back to task manager
// TODO: we can add a separate function in StreamTask to just flush and not return offsets
currentTask.prepareCommit();
// flush the task before giving it back to task manager, if we are not handing it back because of an error.
@ -183,6 +208,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -183,6 +208,7 @@ public class DefaultTaskExecutor implements TaskExecutor {
@ -196,6 +222,8 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -196,6 +222,8 @@ public class DefaultTaskExecutor implements TaskExecutor {
@ -213,10 +241,20 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -213,10 +241,20 @@ public class DefaultTaskExecutor implements TaskExecutor {
thrownewStreamsException("State updater thread did not shutdown within the timeout");
@ -237,8 +275,18 @@ public class DefaultTaskExecutor implements TaskExecutor {
@@ -237,8 +275,18 @@ public class DefaultTaskExecutor implements TaskExecutor {
@ -106,7 +105,8 @@ public class DefaultTaskManager implements TaskManager {
@@ -106,7 +105,8 @@ public class DefaultTaskManager implements TaskManager {
@ -117,6 +117,8 @@ public class DefaultTaskManager implements TaskManager {
@@ -117,6 +117,8 @@ public class DefaultTaskManager implements TaskManager {
}
}
log.debug("Found no assignable task for executor {}",executor.name());
returnnull;
});
}
@ -127,7 +129,8 @@ public class DefaultTaskManager implements TaskManager {
@@ -127,7 +129,8 @@ public class DefaultTaskManager implements TaskManager {
log.debug("Await unblocked: returning early from await since a processable task {} was found",task.id());
returnfalse;
@ -151,7 +154,7 @@ public class DefaultTaskManager implements TaskManager {
@@ -151,7 +154,7 @@ public class DefaultTaskManager implements TaskManager {
@ -177,10 +180,16 @@ public class DefaultTaskManager implements TaskManager {
@@ -177,10 +180,16 @@ public class DefaultTaskManager implements TaskManager {
@ -195,12 +204,18 @@ public class DefaultTaskManager implements TaskManager {
@@ -195,12 +204,18 @@ public class DefaultTaskManager implements TaskManager {
// It can happen that the executor handed back the task before we asked it to
// in which case `streamTask` will be null here.
assertstreamTask==null||streamTask.id()==taskId;
remainingTaskIds.remove(taskId);
if(remainingTaskIds.isEmpty()){
result.complete(null);
}
@ -227,6 +242,11 @@ public class DefaultTaskManager implements TaskManager {
@@ -227,6 +242,11 @@ public class DefaultTaskManager implements TaskManager {
@Override
publicvoidunlockTasks(finalSet<TaskId>taskIds){
if(taskIds.isEmpty()){
return;
}
executeWithTasksLocked(()->{
lockedTasks.removeAll(taskIds);
log.debug("Waking up task executors");
@ -306,11 +326,17 @@ public class DefaultTaskManager implements TaskManager {
@@ -306,11 +326,17 @@ public class DefaultTaskManager implements TaskManager {
@ -334,5 +360,21 @@ public class DefaultTaskManager implements TaskManager {
@@ -334,5 +360,21 @@ public class DefaultTaskManager implements TaskManager {