Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-07 Thread via GitHub


cadonna merged PR #15870:
URL: https://github.com/apache/kafka/pull/15870


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-07 Thread via GitHub


lucasbru commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591960681


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);
+} else {
+tasksToCloseCleanFromStateUpdater.add(task);
+}
+});
+}
+
+private void iterateAndActOnFuture(final Map> futures,
+   final 
java.util.function.Consumer action) {
+for (final Map.Entry> entry : futures.entrySet()) {
+final TaskId taskId = entry.getKey();
+final CompletableFuture future = 
entry.getValue();
+try {
+final StateUpdater.RemovedTaskResult removedTaskResult = 
waitForFuture(taskId, future);
+action.accept(removedTaskResult);
+} catch (final ExecutionException executionException) {
+log.warn("An exception happened when removing task {} from the 
state updater. The exception will be handled later: ",
+taskId, executionException);
+} catch (final InterruptedException ignored) { }

Review Comment:
   Sounds good!



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591277510


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);
+} else {
+tasksToCloseCleanFromStateUpdater.add(task);
+}
+});
+}
+
+private void iterateAndActOnFuture(final Map> futures,
+   final 
java.util.function.Consumer action) {
+for (final Map.Entry> entry : futures.entrySet()) {
+final TaskId taskId = entry.getKey();
+final CompletableFuture future = 
entry.getValue();
+try {
+final StateUpdater.RemovedTaskResult removedTaskResult = 
waitForFuture(taskId, future);
+action.accept(removedTaskResult);
+} catch (final ExecutionException executionException) {
+log.warn("An exception happened when removing task {} from the 
state updater. The exception will be handled later: ",
+taskId, executionException);
+} catch (final InterruptedException ignored) { }

Review Comment:
   I am actually in favor of treating them as fatal and throw an 
`IllegalStateException` to make it more explicit that interruption of a stream 
thread should not happen. WDYT?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591262531


##
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##
@@ -1421,15 +1422,20 @@ public void 
shouldRemoveAllActiveTasksFromStateUpdaterOnPartitionLost() {
 .withInputPartitions(taskId02Partitions).build();
 final TasksRegistry tasks = mock(TasksRegistry.class);
 final TaskManager taskManager = setupForRevocationAndLost(mkSet(task1, 
task2, task3), tasks);
+final CompletableFuture future1 = new 
CompletableFuture<>();

Review Comment:
   Let me write one.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591261680


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);
+} else {
+tasksToCloseCleanFromStateUpdater.add(task);
+}
+});
+}
+
+private void iterateAndActOnFuture(final Map> futures,
+   final 
java.util.function.Consumer action) {
+for (final Map.Entry> entry : futures.entrySet()) {
+final TaskId taskId = entry.getKey();
+final CompletableFuture future = 
entry.getValue();
+try {
+final StateUpdater.RemovedTaskResult removedTaskResult = 
waitForFuture(taskId, future);
+action.accept(removedTaskResult);
+} catch (final ExecutionException executionException) {
+log.warn("An exception happened when removing task {} from the 
state updater. The exception will be handled later: ",
+taskId, executionException);
+} catch (final InterruptedException ignored) { }

Review Comment:
   We are quite inconsistent on how we treat `InterruptedException`. In some 
places we ignore them because they should not happen and in others we treat 
them as fatal and throw an `IllegalStateException` because they should not 
happen [1].
   
   [1] 
https://github.com/apache/kafka/blob/b36cf4ef977fb14bc57683630a9f3f3680705550/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java#L597
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591223152


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);

Review Comment:
   Ah, OK, yes they are logged in the `DefaultStateUpdater` when they happen.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


lucasbru commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591216607


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);

Review Comment:
   I said logging, not throwing. The error should be logged somewhere, right?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591204696


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);
+} else {
+tasksToCloseCleanFromStateUpdater.add(task);
+}
+});
+}
+
+private void iterateAndActOnFuture(final Map> futures,
+   final 
java.util.function.Consumer action) {
+for (final Map.Entry> entry : futures.entrySet()) {
+final TaskId taskId = entry.getKey();
+final CompletableFuture future = 
entry.getValue();
+try {
+final StateUpdater.RemovedTaskResult removedTaskResult = 
waitForFuture(taskId, future);
+action.accept(removedTaskResult);
+} catch (final ExecutionException executionException) {
+log.warn("An exception happened when removing task {} from the 
state updater. The exception will be handled later: ",
+taskId, executionException);
+} catch (final InterruptedException ignored) { }

Review Comment:
   Let me check...



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591204373


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);

Review Comment:
   I thought that these tasks are lost anyways, so why should we bother 
throwing an exception. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591198311


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -71,6 +73,10 @@
 import static 
org.apache.kafka.streams.processor.internals.StateManagerUtil.parseTaskDirectoryName;
 
 public class TaskManager {
+
+private final static String BUG_ERROR_MESSAGE = "This indicates a bug. " +

Review Comment:
   No, we do not have that yet, but I had the same thought. I will look for a 
good place after these PRs are merged.
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


lucasbru commented on code in PR #15870:
URL: https://github.com/apache/kafka/pull/15870#discussion_r1591121181


##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);
+} else {
+tasksToCloseCleanFromStateUpdater.add(task);
+}
+});
+}
+
+private void iterateAndActOnFuture(final Map> futures,
+   final 
java.util.function.Consumer action) {
+for (final Map.Entry> entry : futures.entrySet()) {
+final TaskId taskId = entry.getKey();
+final CompletableFuture future = 
entry.getValue();
+try {
+final StateUpdater.RemovedTaskResult removedTaskResult = 
waitForFuture(taskId, future);
+action.accept(removedTaskResult);
+} catch (final ExecutionException executionException) {
+log.warn("An exception happened when removing task {} from the 
state updater. The exception will be handled later: ",
+taskId, executionException);
+} catch (final InterruptedException ignored) { }

Review Comment:
   Can we just ignore this? I see other classes in the package rethrowing it



##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -71,6 +73,10 @@
 import static 
org.apache.kafka.streams.processor.internals.StateManagerUtil.parseTaskDirectoryName;
 
 public class TaskManager {
+
+private final static String BUG_ERROR_MESSAGE = "This indicates a bug. " +

Review Comment:
   I see that's already defined in a couple of places. Did you check if there 
is a good utility class where this could be defined?



##
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java:
##
@@ -602,6 +608,47 @@ private void removeUnusedTaskFromStateUpdater(final TaskId 
taskId) {
 tasks.addPendingTaskToCloseClean(taskId);
 }
 
+private void addToTasksToClose(final Map> futures,
+   final Set 
tasksToCloseCleanFromStateUpdater,
+   final Set 
tasksToCloseDirtyFromStateUpdater) {
+iterateAndActOnFuture(futures, removedTaskResult -> {
+final Task task = removedTaskResult.task();
+final Optional exception = 
removedTaskResult.exception();
+if (exception.isPresent()) {
+tasksToCloseDirtyFromStateUpdater.add(task);

Review Comment:
   Is the exception already logged somehwere else? We are just dropping it here.



##
streams/src/test/java/org/apache/kafka/streams/processor/internals/TaskManagerTest.java:
##
@@ -1421,15 +1422,20 @@ public void 
shouldRemoveAllActiveTasksFromStateUpdaterOnPartitionLost() {
 .withInputPartitions(taskId02Partitions).build();
 final TasksRegistry tasks = mock(TasksRegistry.class);
 final TaskManager taskManager = setupForRevocationAndLost(mkSet(task1, 
task2, task3), tasks);
+final CompletableFuture future1 = new 
CompletableFuture<>();

Review Comment:
   Do we need a test that covers the part where we get an exception?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org



Re: [PR] KAFKA-10199: Remove lost tasks in state updater with new remove [kafka]

2024-05-06 Thread via GitHub


cadonna closed pull request #15867: KAFKA-10199: Remove lost tasks in state 
updater with new remove
URL: https://github.com/apache/kafka/pull/15867


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org