wcarlson5 commented on a change in pull request #11381:
URL: https://github.com/apache/kafka/pull/11381#discussion_r722408690



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/TaskManager.java
##########
@@ -1280,29 +1281,36 @@ int process(final int maxNumRecords, final Time time) {
             int processed = 0;
             final long then = now;
             try {
-                while (processed < maxNumRecords && task.process(now)) {
-                    task.clearTaskTimeout();
-                    processed++;
+                try {
+                    while (processed < maxNumRecords && task.process(now)) {
+                        task.clearTaskTimeout();
+                        processed++;
+                    }
+                } catch (final TimeoutException timeoutException) {
+                    task.maybeInitTaskTimeoutOrThrow(now, timeoutException);
+                    log.debug(
+                        String.format(
+                            "Could not complete processing records for %s due 
to the following exception; will move to next task and retry later",
+                            task.id()),
+                        timeoutException
+                    );
+                } catch (final TaskMigratedException e) {
+                    log.info("Failed to process stream task {} since it got 
migrated to another thread already. " +
+                                 "Will trigger a new rebalance and close all 
tasks as zombies together.", task.id());
+                    throw e;
+                } catch (final RuntimeException e) {
+                    log.error("Failed to process stream task {} due to the 
following error:", task.id(), e);
+                    throw e;
+                } finally {
+                    now = time.milliseconds();
+                    totalProcessed += processed;
+                    task.recordProcessBatchTime(now - then);
+                }
+            } catch (final Throwable e) {
+                final String topologyName = task.id().topologyName();

Review comment:
       Do we want to swallow if it is not a named topology? should we not just 
rethrow the error in that case?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -299,6 +301,7 @@ public boolean isRunning() {
     private final java.util.function.Consumer<Long> cacheResizer;
 
     private java.util.function.Consumer<Throwable> 
streamsUncaughtExceptionHandler;
+    private final Map<String, java.util.function.Consumer<Throwable>> 
topologyExceptionHandlers;

Review comment:
       Do we want to be able to clean these up after we removing a topology? We 
might not but it might also be a good idea.
   
   Example we remove a topology with a handler and replace one with the same 
name without a handler or using the non named topology handler




-- 
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


Reply via email to