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
Kafka Streams Threading: Exception handling #13957
Changes from 1 commit
35fe831
f160b52
c8536b5
501fa08
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -16,7 +16,9 @@ | |
*/ | ||
package org.apache.kafka.streams.processor.internals.tasks; | ||
|
||
import java.util.Map; | ||
import org.apache.kafka.common.KafkaFuture; | ||
import org.apache.kafka.streams.errors.StreamsException; | ||
import org.apache.kafka.streams.processor.TaskId; | ||
import org.apache.kafka.streams.processor.internals.ReadOnlyTask; | ||
import org.apache.kafka.streams.processor.internals.StreamTask; | ||
|
@@ -98,4 +100,25 @@ public interface TaskManager { | |
* @return set of all managed active tasks | ||
*/ | ||
Set<ReadOnlyTask> getTasks(); | ||
|
||
/** | ||
* Called whenever an existing task has thrown an uncaught exception. | ||
* | ||
* Setting an uncaught exception for a task prevents it from being reassigned until the | ||
* corresponding exception has been handled in the polling thread. | ||
Comment on lines
+107
to
+108
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. That part is not yet implemented, right? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Correct |
||
* | ||
*/ | ||
void setUncaughtException(StreamsException exception, TaskId taskId); | ||
|
||
/** | ||
* Returns and clears all uncaught exceptions that were fell through to the processing | ||
* threads and need to be handled in the polling thread. | ||
* | ||
* Called by the polling thread to handle processing exceptions, e.g. to abort | ||
* transactions or shut down the application. | ||
* | ||
* @return A map from task ID to the exception that occurred. | ||
*/ | ||
Map<TaskId, StreamsException> drainUncaughtExceptions(); | ||
|
||
} |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -21,6 +21,7 @@ | |
import org.apache.kafka.common.utils.MockTime; | ||
import org.apache.kafka.common.utils.Time; | ||
import org.apache.kafka.streams.StreamsConfig; | ||
import org.apache.kafka.streams.errors.StreamsException; | ||
import org.apache.kafka.streams.processor.TaskId; | ||
import org.apache.kafka.streams.processor.internals.StreamTask; | ||
import org.apache.kafka.streams.processor.internals.TasksRegistry; | ||
|
@@ -50,6 +51,7 @@ public class DefaultTaskManagerTest { | |
private final StreamTask task = mock(StreamTask.class); | ||
private final TasksRegistry tasks = mock(TasksRegistry.class); | ||
private final TaskExecutor taskExecutor = mock(TaskExecutor.class); | ||
private final StreamsException exception = mock(StreamsException.class); | ||
|
||
private final StreamsConfig config = new StreamsConfig(configProps()); | ||
private final TaskManager taskManager = new DefaultTaskManager(time, "TaskManager", tasks, config, | ||
|
@@ -166,6 +168,44 @@ public void shouldNotAssignAnyLockedTask() { | |
assertNull(taskManager.assignNextTask(taskExecutor)); | ||
} | ||
|
||
@Test | ||
public void shouldNotSetUncaughtExceptionsForUnassignedTasks() { | ||
taskManager.add(Collections.singleton(task)); | ||
|
||
assertThrows(IllegalArgumentException.class, () -> taskManager.setUncaughtException(exception, task.id())); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you please also verify the error message? Otherwise, we actually do not know which There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done |
||
} | ||
|
||
@Test | ||
public void shouldNotSetUncaughtExceptionsTwice() { | ||
taskManager.add(Collections.singleton(task)); | ||
when(tasks.activeTasks()).thenReturn(Collections.singleton(task)); | ||
taskManager.assignNextTask(taskExecutor); | ||
taskManager.setUncaughtException(exception, task.id()); | ||
|
||
assertThrows(IllegalArgumentException.class, () -> taskManager.setUncaughtException(exception, task.id())); | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Could you please also verify the error message? Otherwise, we actually do not know which There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done |
||
} | ||
|
||
@Test | ||
public void shouldReturnExceptionsOnDrainExceptions() { | ||
taskManager.add(Collections.singleton(task)); | ||
when(tasks.activeTasks()).thenReturn(Collections.singleton(task)); | ||
taskManager.assignNextTask(taskExecutor); | ||
taskManager.setUncaughtException(exception, task.id()); | ||
|
||
assertEquals(taskManager.drainUncaughtExceptions(), Collections.singletonMap(task.id(), exception)); | ||
} | ||
|
||
@Test | ||
public void shouldClearExceptionsOnDrainExceptions() { | ||
taskManager.add(Collections.singleton(task)); | ||
when(tasks.activeTasks()).thenReturn(Collections.singleton(task)); | ||
taskManager.assignNextTask(taskExecutor); | ||
taskManager.setUncaughtException(exception, task.id()); | ||
taskManager.drainUncaughtExceptions(); | ||
|
||
assertEquals(taskManager.drainUncaughtExceptions(), Collections.emptyMap()); | ||
} | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I think, it would be OK to merge these two tests. Just add the There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Done |
||
|
||
@Test | ||
public void shouldUnassignLockingTask() { | ||
final KafkaFutureImpl<StreamTask> future = new KafkaFutureImpl<>(); | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we log some more information about the exception, like the type and the error message? Just to be able to better reason about the error case.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Done