ableegoldman commented on a change in pull request #9487:
URL: https://github.com/apache/kafka/pull/9487#discussion_r518478117



##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -567,10 +589,34 @@ void runLoop() {
                 }
             } catch (final TaskMigratedException e) {
                 handleTaskMigrated(e);
+            } catch (final Exception e) {
+                if (this.streamsUncaughtExceptionHandler.handle(e)) {

Review comment:
       I had a little trouble following the `Handler` class. Some trivial 
things -- eg the handler in the StreamThread is named 
`streamsUncaughtExceptionHandler` but it's actually _not_ a 
`StreamsUncaughtExceptionHandler`. Also the usage of the return value; IIUC 
it's supposed to indicate whether to use the new handler or fall back on the 
old one. To me it sounds like if `handle` returns `true` that means we should 
handle it, ie we should _not_ rethrow the exception, but this looks like the 
opposite of what we do now. Honestly either interpretation is ok with me, as 
long as it's documented somewhere
   
   Do we really need the `Handler` in the first place though? It's already 
pretty confusing that we have to deal with two types of handlers (old and new) 
so I'd prefer not to add a third unless it's really necessary. It seems like we 
can just inline the logic of whether to invoke the new handler or rethrow the 
exception, which would also clear up the confusion around the meaning of the 
return value. But I might be missing something here -- WDYT?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -282,6 +283,17 @@ public boolean isRunning() {
     private final Admin adminClient;
     private final InternalTopologyBuilder builder;
 
+    private Handler streamsUncaughtExceptionHandler;
+    private ShutdownErrorHook shutdownErrorHook;
+    private AtomicInteger assignmentErrorCode;
+    public interface ShutdownErrorHook {
+        void shutdown();
+    }

Review comment:
       Seems like we can just pass in a Runnable with 
`KafkaStreams::closeToError` instead of adding a whole `ShutdownErrorHook` 
functional interface

##########
File path: streams/src/main/resources/common/message/SubscriptionInfoData.json
##########
@@ -57,6 +57,11 @@
       "name": "uniqueField",
       "versions": "8+",
       "type": "int8"
+    },
+    {
+      "name": "shutdownRequested",
+      "versions": "9+",
+      "type": "int8"

Review comment:
       I think we should mirror the `errorCode` in the AssignmentInfo here, 
both in terms of naming and type. If we're going to use the same AssignorError 
for both, then they should really be the same. And we may want to send other 
kinds of error codes in the subscription going forward: better to just encode a 
single `int` than a separate `byte` for every logical error code. I don't think 
we'll notice the extra three bytes since Subscriptions aren't sent that 
frequently

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/ReferenceContainer.java
##########
@@ -30,7 +30,7 @@
     public Admin adminClient;
     public TaskManager taskManager;
     public StreamsMetadataState streamsMetadataState;
-    public final AtomicInteger assignmentErrorCode = new AtomicInteger();
+    public AtomicInteger assignmentErrorCode = new AtomicInteger();

Review comment:
       This should probably stay `final` so we don't accidentally change it ever

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());

Review comment:
       Should this be logged at error?

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -996,6 +1082,62 @@ private boolean close(final long timeoutMs) {
         }
     }
 
+    private void closeToError() {
+        if (!setState(State.ERROR)) {
+            log.info("Can not transition to error from state " + state());
+        } else {
+            log.info("Transitioning to ERROR state");
+            stateDirCleaner.shutdownNow();
+            if (rocksDBMetricsRecordingService != null) {
+                rocksDBMetricsRecordingService.shutdownNow();
+            }
+
+            // wait for all threads to join in a separate thread;
+            // save the current thread so that if it is a stream thread
+            // we don't attempt to join it and cause a deadlock
+            final Thread shutdownThread = new Thread(() -> {
+                // notify all the threads to stop; avoid deadlocks by stopping 
any
+                // further state reports from the thread since we're shutting 
down
+                for (final StreamThread thread : threads) {
+                    thread.shutdown();
+                }
+
+                for (final StreamThread thread : threads) {
+                    try {
+                        if (!thread.isRunning()) {
+                            thread.join();
+                        }
+                    } catch (final InterruptedException ex) {
+                        Thread.currentThread().interrupt();
+                    }
+                }
+
+                if (globalStreamThread != null) {
+                    globalStreamThread.shutdown();
+                }
+
+                if (globalStreamThread != null && 
!globalStreamThread.stillRunning()) {
+                    try {
+                        globalStreamThread.join();
+                    } catch (final InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                    }
+                    globalStreamThread = null;
+                }
+
+                adminClient.close();
+
+                streamsMetrics.removeAllClientLevelMetrics();
+                metrics.close();
+                setState(State.ERROR);
+            }, "kafka-streams-close-thread");
+
+            shutdownThread.setDaemon(true);
+            shutdownThread.start();
+            setState(State.ERROR);

Review comment:
       Looks like we call `setState(ERROR)` three times in this method, is that 
intentional?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##########
@@ -567,10 +589,34 @@ void runLoop() {
                 }
             } catch (final TaskMigratedException e) {
                 handleTaskMigrated(e);
+            } catch (final Exception e) {
+                if (this.streamsUncaughtExceptionHandler.handle(e)) {
+                    throw e;
+                }
             }
         }
     }
 
+    /**
+     * Sets the streams uncaught exception handler.
+     *
+     * @param streamsUncaughtExceptionHandler the user handler wrapped in 
shell to execute the action
+     */
+    public void setStreamsUncaughtExceptionHandler(final Handler 
streamsUncaughtExceptionHandler) {
+        this.streamsUncaughtExceptionHandler = streamsUncaughtExceptionHandler;
+    }
+
+    public void shutdownToError() {
+        shutdownErrorHook.shutdown();
+    }
+
+    public void sendShutdownRequest(final AssignorError assignorError) {
+        log.warn("Detected that shutdown was requested. " +
+                "The all clients in this app will now begin to shutdown");

Review comment:
       ```suggestion
                   "All clients in this app will now begin to shutdown");
   ```

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsPartitionAssignor.java
##########
@@ -255,8 +255,9 @@ public ByteBuffer subscriptionUserData(final Set<String> 
topics) {
             taskManager.processId(),
             userEndPoint,
             taskManager.getTaskOffsetSums(),
-            uniqueField)
-                .encode();
+            uniqueField,
+            (byte) assignmentErrorCode.get()

Review comment:
       This cast makes me kind of uncomfortable...either the 
`assignmentErrorCode` that we have in the AssignmentInfo is conceptually the 
same as the one we're adding to the SubscriptionInfo (in which case it should 
be the same type), or it's not the same, in which case we should use a 
different variable to track it.
   
   Personally I think it's probably simpler to keep them the same, and just add 
an `int` errorCode field to the Subscription instead of a `byte` 
shutdownRequested field. 

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
##########
@@ -60,6 +60,11 @@ public void onPartitionsAssigned(final 
Collection<TopicPartition> partitions) {
         }  else if (assignmentErrorCode.get() == 
AssignorError.ASSIGNMENT_ERROR.code()) {
             log.error("Received error code {}", 
AssignorError.ASSIGNMENT_ERROR);
             throw new TaskAssignmentException("Hit an unexpected exception 
during task assignment phase of rebalance");
+        } else if (assignmentErrorCode.get() == 
AssignorError.SHUTDOWN_REQUESTED.code()) {
+            log.error("A Kafka Streams client in this Kafka Streams 
application is requesting to shutdown the application");
+            streamThread.shutdownToError();
+            taskManager.handleRebalanceComplete();

Review comment:
       It probably doesn't matter too much since `handleRebalanceComplete` 
doesn't do anything that important at the mometn, but it seems like we should 
call it before shutting down, not after.

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -366,6 +373,84 @@ public void setUncaughtExceptionHandler(final 
Thread.UncaughtExceptionHandler eh
         }
     }
 
+    /**
+     * Set the handler invoked when an {@link 
StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread}
+     * throws an unexpected exception.
+     * These might be exceptions indicating rare bugs in Kafka Streams, or they
+     * might be exceptions thrown by your code, for example a 
NullPointerException thrown from your processor
+     * logic.
+     * <p>
+     * Note, this handler must be threadsafe, since it will be shared among 
all threads, and invoked from any
+     * thread that encounters such an exception.
+     *
+     * @param streamsUncaughtExceptionHandler the uncaught exception handler 
of type {@link StreamsUncaughtExceptionHandler} for all internal threads
+     * @throws IllegalStateException if this {@code KafkaStreams} instance is 
not in state {@link State#CREATED CREATED}.
+     * @throws NullPointerException if streamsUncaughtExceptionHandler is null.
+     */
+    public void setUncaughtExceptionHandler(final 
StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamThread.Handler handler = exception -> 
handleStreamsUncaughtException(exception, streamsUncaughtExceptionHandler);
+        synchronized (stateLock) {
+            if (state == State.CREATED) {
+                Objects.requireNonNull(streamsUncaughtExceptionHandler);
+                for (final StreamThread thread : threads) {
+                    thread.setStreamsUncaughtExceptionHandler(handler);
+                }
+                if (globalStreamThread != null) {
+                    globalStreamThread.setUncaughtExceptionHandler(handler);
+                }
+            } else {
+                throw new IllegalStateException("Can only set 
UncaughtExceptionHandler in CREATED state. " +
+                        "Current state is: " + state);
+            }
+        }
+    }
+
+    private boolean handleStreamsUncaughtExceptionDefaultWrapper(final 
Throwable throwable) {
+        if (Thread.getDefaultUncaughtExceptionHandler() != null) {
+            return true;
+        }
+        return handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT);
+    }
+
+    private boolean handleStreamsUncaughtException(final Throwable e,
+                                                   final 
StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+        final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse 
action = streamsUncaughtExceptionHandler.handle(e);
+        if (Thread.getDefaultUncaughtExceptionHandler() != null) {
+            log.warn("Stream's new uncaught exception handler is set as well 
as the deprecated old handler." +
+                    "The old handler will be ignored as long as a new handler 
is set.");
+        }
+        switch (action) {
+            case SHUTDOWN_CLIENT:
+                log.error("Encountered the following exception during 
processing " +
+                        "and the registered exception handler opted to \" + 
action + \"." +
+                        " The streams client is going to shut down now. ", e);
+                close(Duration.ZERO);
+                break;
+            case SHUTDOWN_APPLICATION:
+                if (e instanceof Error) {
+                    log.error("This option requires running threads to shut 
down the application." +
+                            "but the uncaught exception was an Error, which 
means this runtime is no " +
+                            "longer in a well-defined state. Attempting to 
send the shutdown command anyway.", e);
+                }
+                if (Thread.currentThread().equals(globalStreamThread) && 
threads.stream().noneMatch(StreamThread::isRunning)) {
+                    log.error("Exception in global stream thread cause the 
application to attempt to shutdown." +
+                            " This action will succeed only if there is at 
least one StreamThread running on ths client." +
+                            " Currently there is no running threads so will 
now close the client.");

Review comment:
       ```suggestion
                       log.error("Exception in global thread caused the 
application to attempt to shutdown." +
                               " This action will succeed only if there is at 
least one StreamThread running on this client." +
                               " Currently there are no running threads so will 
now close the client.");
   ```




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

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


Reply via email to