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



##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -971,6 +972,9 @@ private StreamThread createAndAddStreamThread(final long 
cacheSizePerThread, fin
             streamsUncaughtExceptionHandler
         );
         streamThread.setStateListener(streamStateListener);
+        for (final Map.Entry<String, Consumer<Throwable>> exceptionHandler : 
topologyExceptionHandlers.entrySet()) {

Review comment:
       Should we synchronize on the `changeThreadCount` object as well?

##########
File path: 
streams/src/main/java/org/apache/kafka/streams/errors/NamedTopologyException.java
##########
@@ -0,0 +1,33 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.kafka.streams.errors;
+
+import org.apache.kafka.common.KafkaException;
+
+public class NamedTopologyException extends KafkaException {
+
+    final String topologyName;
+
+    public NamedTopologyException(final String topologyName, final Throwable 
throwable) {

Review comment:
       Could we consider overriding the `toString` function to include the 
`topologyName`?

##########
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##########
@@ -513,8 +514,8 @@ private boolean wrappedExceptionIsIn(final Throwable 
throwable, final Set<Class<
         return action;
     }
 
-    private void handleStreamsUncaughtException(final Throwable throwable,
-                                                final 
StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+    protected void handleStreamsUncaughtException(final Throwable throwable,

Review comment:
       Since we wrap the real exception with the NamedTopologyException, the 
logic inside `getActionForThrowable` need to possible peal off that layer (i.e. 
we may need to unwrap twice)..
   
   I'm thinking if we can do something simpler than wrapping / unwrapping here: 
   1) we are almost wrapping all non-streams exceptions as a StreamsException 
anyways, so let's just take a look at where we do not and enforce that. In a 
way we know that the thrown exception to the thread is either a 
StreamsException by itself and has no cause, or a wrapped StreamsException (but 
only wrapped once) with a cause.
   2) We added a `topologyName` to StreamsException, where `null` indicate it's 
global. When throwing that directly or wrapping it on a non-streams cause, we 
set this field.
   3) The uncaughtExceptionHandler would then expect only only see 
`StreamsException` as in 1) above or unchecked exception; for the former we 
look at that field directly, for the later we just treat it as a global fatal 
one.
   
   Then we do not need a wrapper exception anymore, and does not need nested 
try-catch either.




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