[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-18 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, 
final TimeUnit timeUnit) {
 return close(timeoutMs);
 }
 
-private boolean close(final long timeoutMs) {
-if (!setState(State.PENDING_SHUTDOWN)) {
-// if transition failed, it means it was either in PENDING_SHUTDOWN
-// or NOT_RUNNING already; just check that all threads have been 
stopped
-log.info("Already in the pending shutdown state, wait to complete 
shutdown");
-} else {
-stateDirCleaner.shutdownNow();
-if (rocksDBMetricsRecordingService != null) {
-rocksDBMetricsRecordingService.shutdownNow();
-}
+private Thread shutdownHelper(final boolean error) {
+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();
-}
+// 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
+return 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();
+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.shutdown();
+}
 
-if (globalStreamThread != null && 
!globalStreamThread.stillRunning()) {
-try {
-globalStreamThread.join();
-} catch (final InterruptedException e) {
-Thread.currentThread().interrupt();
-}
-globalStreamThread = null;
+if (globalStreamThread != null && 
!globalStreamThread.stillRunning()) {
+try {
+globalStreamThread.join();
+} catch (final InterruptedException e) {
+Thread.currentThread().interrupt();
 }
+globalStreamThread = null;
+}
 
-adminClient.close();
+adminClient.close();
 
-streamsMetrics.removeAllClientLevelMetrics();
-metrics.close();
+streamsMetrics.removeAllClientLevelMetrics();
+metrics.close();
+if (!error) {
 setState(State.NOT_RUNNING);
-}, "kafka-streams-close-thread");
+}
+}, "kafka-streams-close-thread");
+}
+
+private boolean close(final long timeoutMs) {
+if (!setState(State.PENDING_SHUTDOWN)) {

Review comment:
   I do think that Error should not have direct transition. However I don't 
like using `PENDING_SHUTDOWN` , mostly because we can already distinguish 
between the two states and it would be best to inform right away. Also it could 
be a problem if we went to set Error and some how it went from PENDING_SHUTDOWN 
to NOT_RUNNING. I am in favor of adding something like `PENDING_ERROR` just to 
be more precise. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to 

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-17 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
##
@@ -311,6 +314,8 @@ public void run() {
 "Updating global state failed. You can restart KafkaStreams to 
recover from this error.",
 recoverableException

Review comment:
   like in stream thread we can just add a call to the 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.

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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-17 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, 
final TimeUnit timeUnit) {
 return close(timeoutMs);
 }
 
-private boolean close(final long timeoutMs) {
-if (!setState(State.PENDING_SHUTDOWN)) {
-// if transition failed, it means it was either in PENDING_SHUTDOWN
-// or NOT_RUNNING already; just check that all threads have been 
stopped
-log.info("Already in the pending shutdown state, wait to complete 
shutdown");
-} else {
-stateDirCleaner.shutdownNow();
-if (rocksDBMetricsRecordingService != null) {
-rocksDBMetricsRecordingService.shutdownNow();
-}
+private Thread shutdownHelper(final boolean error) {
+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();
-}
+// 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
+return 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();
+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.shutdown();
+}
 
-if (globalStreamThread != null && 
!globalStreamThread.stillRunning()) {
-try {
-globalStreamThread.join();
-} catch (final InterruptedException e) {
-Thread.currentThread().interrupt();
-}
-globalStreamThread = null;
+if (globalStreamThread != null && 
!globalStreamThread.stillRunning()) {
+try {
+globalStreamThread.join();
+} catch (final InterruptedException e) {
+Thread.currentThread().interrupt();
 }
+globalStreamThread = null;
+}
 
-adminClient.close();
+adminClient.close();
 
-streamsMetrics.removeAllClientLevelMetrics();
-metrics.close();
+streamsMetrics.removeAllClientLevelMetrics();
+metrics.close();
+if (!error) {
 setState(State.NOT_RUNNING);
-}, "kafka-streams-close-thread");
+}
+}, "kafka-streams-close-thread");
+}
+
+private boolean close(final long timeoutMs) {
+if (!setState(State.PENDING_SHUTDOWN)) {

Review comment:
   This is currently the plan to remove that transition. It is pretty much 
the only change we plan to make to the FSM.





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-17 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +377,90 @@ 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.
+ * The handler will execute on the thread that produced the exception.
+ * In order to get the thread that threw the exception, 
Thread.currentThread().
+ * 
+ * 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 Consumer 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 void defaultStreamsUncaughtExceptionHandler(final Throwable 
throwable) {
+if (oldHandler) {
+if (throwable instanceof RuntimeException) {
+throw (RuntimeException) throwable;
+} else if (throwable instanceof Error) {
+throw (Error) throwable;
+} else {
+throw new RuntimeException("Unexpected checked exception 
caught in the uncaught exception handler", throwable);
+}
+} else {
+handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT);
+}
+}
+
+private void handleStreamsUncaughtException(final Throwable throwable,
+final 
StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse 
action = streamsUncaughtExceptionHandler.handle(throwable);
+if (oldHandler) {
+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. ", 
throwable);
+close(Duration.ZERO);
+break;
+case SHUTDOWN_APPLICATION:
+if (throwable 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.", throwable);
+}
+if (Thread.currentThread().equals(globalStreamThread) && 
threads.stream().noneMatch(StreamThread::isRunning)) {
+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.");
+close(Duration.ZERO);

Review comment:
   I am on the fence about this. I do think its would be consistent to be 
not running but also it did 

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-17 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -932,56 +1028,62 @@ public synchronized boolean close(final long timeout, 
final TimeUnit timeUnit) {
 return close(timeoutMs);
 }
 
-private boolean close(final long timeoutMs) {
-if (!setState(State.PENDING_SHUTDOWN)) {
-// if transition failed, it means it was either in PENDING_SHUTDOWN
-// or NOT_RUNNING already; just check that all threads have been 
stopped
-log.info("Already in the pending shutdown state, wait to complete 
shutdown");
-} else {
-stateDirCleaner.shutdownNow();
-if (rocksDBMetricsRecordingService != null) {
-rocksDBMetricsRecordingService.shutdownNow();
-}
+private Thread shutdownHelper(final boolean error) {
+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();
-}
+// 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
+return 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();
+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.shutdown();
+}

Review comment:
   You are right I think. I just copied from the normal close method 
because I knew it worked. In a follow up we can maybe change both of these. Do 
you think that there should be a ak ticket to track it?





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-17 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +377,90 @@ 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.
+ * The handler will execute on the thread that produced the exception.
+ * In order to get the thread use that threw the exception, 
Thread.currentThread().

Review comment:
   need to remove `use`





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-17 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +377,90 @@ 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.
+ * The handler will execute on the thread that produced the exception.
+ * In order to get the thread use that threw the exception, 
Thread.currentThread().
+ * 
+ * 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 Consumer 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 void defaultStreamsUncaughtExceptionHandler(final Throwable 
throwable) {
+if (oldHanlder) {

Review comment:
   oops





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-16 Thread GitBox


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



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

Review comment:
   I guess I must have misunderstood  your earlier comment. I thought you 
wanted it to stay a byte so that is why I pushed back. But if you have no 
objections I will just change it 





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-16 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +377,90 @@ 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.
+ * The handler will execute on the thread that produced the exception.
+ * In order to get the thread use Thread.currentThread()
+ * 
+ * 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 Consumer 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 void defaultStreamsUncaughtExceptionHandler(final Throwable 
throwable) {
+if (oldHanlder) {
+if (throwable instanceof RuntimeException) {
+throw (RuntimeException) throwable;
+} else if (throwable instanceof Error) {
+throw (Error) throwable;
+} else {
+throw new RuntimeException("Unexpected checked exception 
caught in the uncaught exception handler", throwable);
+}
+} else {
+handleStreamsUncaughtException(throwable, t -> SHUTDOWN_CLIENT);
+}
+}
+
+private void handleStreamsUncaughtException(final Throwable throwable,
+final 
StreamsUncaughtExceptionHandler streamsUncaughtExceptionHandler) {
+final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse 
action = streamsUncaughtExceptionHandler.handle(throwable);
+if (oldHanlder) {
+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. ", 
throwable);
+close(Duration.ZERO);
+break;
+case SHUTDOWN_APPLICATION:
+if (throwable 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.", throwable);
+}
+if (Thread.currentThread().equals(globalStreamThread) && 
threads.stream().noneMatch(StreamThread::isRunning)) {
+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.");
+close();

Review comment:
   It doesn't really matter to me, though I think that non blocking is 
probably  preferable.





[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-16 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##
@@ -559,18 +552,52 @@ void runLoop() {
 }
 } catch (final TaskCorruptedException e) {
 log.warn("Detected the states of tasks " + 
e.corruptedTaskWithChangelogs() + " are corrupted. " +
- "Will close the task as dirty and re-create and 
bootstrap from scratch.", e);
+"Will close the task as dirty and re-create and 
bootstrap from scratch.", e);
 try {
 
taskManager.handleCorruption(e.corruptedTaskWithChangelogs());
 } catch (final TaskMigratedException taskMigrated) {
 handleTaskMigrated(taskMigrated);
 }
 } catch (final TaskMigratedException e) {
 handleTaskMigrated(e);
+} catch (final UnsupportedVersionException e) {

Review comment:
   That is probably fine. We can really get into it when we add the replace 
option, as now all calls to the handler are fatal.





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-16 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
##
@@ -311,6 +314,8 @@ public void run() {
 "Updating global state failed. You can restart KafkaStreams to 
recover from this error.",
 recoverableException
 );
+} catch (final Exception e) {
+this.streamsUncaughtExceptionHandler.accept(e);

Review comment:
   +1 to sorting out FSM before next release, I have a ticket to track the 
work. I started to change it and it ballooned out to be much more expansive 
than I thought. This PR is already complicated enough, so we can add is later.





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-16 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
##
@@ -1000,7 +1012,17 @@ public void restore(final Map tasks) {
 CLIENT_ID,
 new LogContext(""),
 new AtomicInteger(),
-new AtomicLong(Long.MAX_VALUE)
+new AtomicLong(Long.MAX_VALUE),
+null,
+e -> {
+if (e instanceof RuntimeException) {
+throw (RuntimeException) e;
+} else if (e instanceof Error) {
+throw (Error) e;
+} else {
+throw new RuntimeException("Unexpected checked exception 
caught in the uncaught exception handler", e);
+}

Review comment:
   The default is in KafkaStreams, but I see your point. We can make all of 
them rethrow then we will not have to worry about swallowing





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-16 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +375,93 @@ 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.
+ * The handler will execute on the thread that produced the exception.
+ * So inorder to get the thread as the java handler type uses use 
Thread.currentThread()
+ * 
+ * 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 Consumer 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 void handleStreamsUncaughtExceptionDefaultWrapper(final Throwable 
throwable) {

Review comment:
   It's actually not always used. It is only used until a new handler is 
set in which it is over written. Once that happens we don't want the old 
handler to be set so we do not wrap a user provided handler with this method





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-13 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/SubscriptionInfo.java
##
@@ -115,6 +119,10 @@ private SubscriptionInfo(final SubscriptionInfoData 
subscriptionInfoData) {
 this.data = subscriptionInfoData;
 }
 
+public int errorCode() {
+return data.errorCode();

Review comment:
   Good idea. It does not seem to do anything. but good to have a test for 
it





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-13 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
##
@@ -1000,7 +1012,17 @@ public void restore(final Map tasks) {
 CLIENT_ID,
 new LogContext(""),
 new AtomicInteger(),
-new AtomicLong(Long.MAX_VALUE)
+new AtomicLong(Long.MAX_VALUE),
+null,
+e -> {
+if (e instanceof RuntimeException) {
+throw (RuntimeException) e;
+} else if (e instanceof Error) {
+throw (Error) e;
+} else {
+throw new RuntimeException("Unexpected checked exception 
caught in the uncaught exception handler", e);
+}

Review comment:
   Not quite. If I remove the handler and just run it there is an illegal 
state exception which runs endlessly until the handler can exit the loop. It 
looks like the thread hadn't started all the way before the 
TaskMigratedExcpetion is thrown
   
   `INFO State transition from STARTING to PENDING_SHUTDOWN 
(org.apache.kafka.streams.processor.internals.StreamThread:223)
   [`





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-13 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionIntegrationTest.java
##
@@ -144,6 +143,7 @@ public void whenShuttingDown() throws IOException {
 }
 
 @Test
+@Deprecated //a single thread should no longer die

Review comment:
   I agree we shouldn't remove the valid test cases. Maybe the ones that 
are more complicated I can just set an idempotent old handler and mark as 
deprecated and we can file tickets to update. Either we work them down or when 
we go to remove the old handler they will fail and we need to fix them then.





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-13 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
##
@@ -311,6 +314,8 @@ public void run() {
 "Updating global state failed. You can restart KafkaStreams to 
recover from this error.",
 recoverableException
 );
+} catch (final Exception e) {
+this.streamsUncaughtExceptionHandler.accept(e);

Review comment:
   I don't think it will actually transition to `ERROR` because the handler 
will call close before the global thread is dead, which will transition to 
PEDING_SHUTDOWN, there is no transition to ERROR from either PENDING_SHUTDOWN 
or NOT_RUNNING.
   
   the FSM will be part of the add thread work as it doesn't really make sense 
to remove the change to error until we can add threads





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-13 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +375,93 @@ 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.
+ * The handler will execute on the thread that produced the exception.
+ * So inorder to get the thread as the java handler type uses use 
Thread.currentThread()

Review comment:
   I appreciate the benefit of the doubt :) but you are right there is an 
extra `uses`





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-13 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +375,93 @@ 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.
+ * The handler will execute on the thread that produced the exception.
+ * So inorder to get the thread as the java handler type uses use 
Thread.currentThread()

Review comment:
   changed to ` In order to get the thread uses use Thread.currentThread()`
   
   Does that work better?

##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
##
@@ -60,6 +60,11 @@ public void onPartitionsAssigned(final 
Collection 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:
   For the same reason I had to add to the other cases as the close from 
the new handler will not finish otherwise

##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +374,63 @@ 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.
+ * 
+ * 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 StreamsUncaughtExceptionHandler 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 StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse 
handleStreamsUncaughtException(final Throwable e,
+   
  final StreamsUncaughtExceptionHandler 
streamsUncaughtExceptionHandler) {
+final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse 
action = streamsUncaughtExceptionHandler.handle(e);
+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);

Review comment:
   we should be able to change it to `close()`

##
File path: 

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-09 Thread GitBox


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



##
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:
   https://github.com/apache/kafka/pull/9273#discussion_r486597512
   I originally had it at int32, but @vvcephei suggested int16, now it is int8.
   
   would you be good with int16 or do you think int32 is the way?





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-09 Thread GitBox


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



##
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:
   https://github.com/apache/kafka/pull/9273#discussion_r486597512
   I originally had it at int32, but john suggested int16, now it is int8.
   
   would you be good with int16 or do you think int32 is the way?





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
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:
   That is a good idea, Ill change the log





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
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 I agree on the name, I am not sure about the type. We should be 
able to fit thousands of different error code into the byte so we should not 
run out of space. The reason the errorCode. is an integer in the first place is 
because there is not `AtomicByte` that I know of.





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
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:
   Yes we can





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
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:
   I was changing it intentionally but I think I can get away with not





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
##
@@ -60,6 +60,11 @@ public void onPartitionsAssigned(final 
Collection 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:
   We can do that, it doesn't seem make difference which order it is 
called. However if it is not called it will get stuck continually rebalancing. 
We return because setting the state to partitions assigned will cause an error





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
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:
   No, I hadn't seen that





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
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:
   In the normal close method the corresponding log is also info. As 
multiple thread will be calling this at once I would rather not flood the logs 
with error unnecessarily.





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-06 Thread GitBox


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



##
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:
   We could do the logic inline how ever this does make it slightly 
simpler. Also we only expose the `streamsUncaughtExceptionHandler` to the user 
and @vvcephei had a problem with the wrapping that again with the same type. So 
we introduced a wrapper class. if we renamed it from `Handler` to 
`streamsUncaughtExceptionHandlerWrapper` would that make it more clear?





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+@Rule
+public TestName testName = new TestName();
+
+String inputTopic;
+StreamsBuilder builder;
+Properties properties;
+List processorValueCollector;
+String idempotentTopic = "idempotentTopic";
+String appId = "";
+
+@Before
+public void setup() {
+final String testId = safeUniqueTestName(getClass(), testName);
+appId = "appId_" + testId;
+inputTopic = "input" + testId;
+cleanStateBeforeTest(CLUSTER, idempotentTopic, inputTopic);
+
+IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);

Review comment:
   good questions





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamsRebalanceListener.java
##
@@ -60,6 +60,9 @@ public void onPartitionsAssigned(final 
Collection 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()) {

Review comment:
   added unit test





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+@Rule
+public TestName testName = new TestName();
+
+String inputTopic;
+StreamsBuilder builder;
+Properties properties;
+List processorValueCollector;
+String idempotentTopic = "idempotentTopic";

Review comment:
   it can be removed





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+@Rule
+public TestName testName = new TestName();
+
+String inputTopic;
+StreamsBuilder builder;
+Properties properties;
+List processorValueCollector;
+String idempotentTopic = "idempotentTopic";
+String appId = "";
+
+@Before
+public void setup() {
+final String testId = safeUniqueTestName(getClass(), testName);
+appId = "appId_" + testId;
+inputTopic = "input" + testId;
+cleanStateBeforeTest(CLUSTER, idempotentTopic, inputTopic);
+
+IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);
+
+builder  = new StreamsBuilder();
+
+processorValueCollector = new ArrayList<>();
+
+final KStream stream = builder.stream(inputTopic);
+stream.process(() -> new ShutdownProcessor(processorValueCollector), 
Named.as("process"));
+
+properties  = mkObjectProperties(
+mkMap(
+mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath()),
+mkEntry(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 
"5"),
+mkEntry(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, 
"6"),
+mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "7"),
+

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+@Rule
+public TestName testName = new TestName();
+
+String inputTopic;
+StreamsBuilder builder;
+Properties properties;
+List processorValueCollector;
+String idempotentTopic = "idempotentTopic";
+String appId = "";
+
+@Before
+public void setup() {
+final String testId = safeUniqueTestName(getClass(), testName);
+appId = "appId_" + testId;
+inputTopic = "input" + testId;
+cleanStateBeforeTest(CLUSTER, idempotentTopic, inputTopic);
+
+IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);
+
+builder  = new StreamsBuilder();
+
+processorValueCollector = new ArrayList<>();
+
+final KStream stream = builder.stream(inputTopic);
+stream.process(() -> new ShutdownProcessor(processorValueCollector), 
Named.as("process"));
+
+properties  = mkObjectProperties(
+mkMap(
+mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath()),
+mkEntry(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 
"5"),
+mkEntry(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, 
"6"),
+mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "7"),
+

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+@Rule
+public TestName testName = new TestName();
+
+String inputTopic;
+StreamsBuilder builder;
+Properties properties;
+List processorValueCollector;
+String idempotentTopic = "idempotentTopic";
+String appId = "";
+
+@Before
+public void setup() {
+final String testId = safeUniqueTestName(getClass(), testName);
+appId = "appId_" + testId;
+inputTopic = "input" + testId;
+cleanStateBeforeTest(CLUSTER, idempotentTopic, inputTopic);
+
+IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);
+
+builder  = new StreamsBuilder();
+
+processorValueCollector = new ArrayList<>();
+
+final KStream stream = builder.stream(inputTopic);
+stream.process(() -> new ShutdownProcessor(processorValueCollector), 
Named.as("process"));
+
+properties  = mkObjectProperties(
+mkMap(
+mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath()),
+mkEntry(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 
"5"),
+mkEntry(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, 
"6"),
+mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "7"),
+

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+@Rule
+public TestName testName = new TestName();
+
+String inputTopic;
+StreamsBuilder builder;
+Properties properties;
+List processorValueCollector;
+String idempotentTopic = "idempotentTopic";
+String appId = "";
+
+@Before
+public void setup() {
+final String testId = safeUniqueTestName(getClass(), testName);
+appId = "appId_" + testId;
+inputTopic = "input" + testId;
+cleanStateBeforeTest(CLUSTER, idempotentTopic, inputTopic);
+
+IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);
+
+builder  = new StreamsBuilder();
+
+processorValueCollector = new ArrayList<>();
+
+final KStream stream = builder.stream(inputTopic);
+stream.process(() -> new ShutdownProcessor(processorValueCollector), 
Named.as("process"));
+
+properties  = mkObjectProperties(
+mkMap(
+mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath()),
+mkEntry(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 
"5"),
+mkEntry(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, 
"6"),
+mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "7"),
+

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {
+@ClassRule
+public static final EmbeddedKafkaCluster CLUSTER = new 
EmbeddedKafkaCluster(1);
+
+@Rule
+public TestName testName = new TestName();
+
+String inputTopic;
+StreamsBuilder builder;
+Properties properties;
+List processorValueCollector;
+String idempotentTopic = "idempotentTopic";
+String appId = "";
+
+@Before
+public void setup() {
+final String testId = safeUniqueTestName(getClass(), testName);
+appId = "appId_" + testId;
+inputTopic = "input" + testId;
+cleanStateBeforeTest(CLUSTER, idempotentTopic, inputTopic);
+
+IntegrationTestUtils.cleanStateBeforeTest(CLUSTER, inputTopic);
+
+builder  = new StreamsBuilder();
+
+processorValueCollector = new ArrayList<>();
+
+final KStream stream = builder.stream(inputTopic);
+stream.process(() -> new ShutdownProcessor(processorValueCollector), 
Named.as("process"));
+
+properties  = mkObjectProperties(
+mkMap(
+mkEntry(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, 
CLUSTER.bootstrapServers()),
+mkEntry(StreamsConfig.APPLICATION_ID_CONFIG, appId),
+mkEntry(StreamsConfig.STATE_DIR_CONFIG, 
TestUtils.tempDirectory().getPath()),
+mkEntry(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 
"5"),
+mkEntry(StreamsConfig.ACCEPTABLE_RECOVERY_LAG_CONFIG, 
"6"),
+mkEntry(StreamsConfig.MAX_WARMUP_REPLICAS_CONFIG, "7"),
+

[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: 
streams/src/test/java/org/apache/kafka/streams/integration/StreamsHandlerIntegrationTest.java
##
@@ -0,0 +1,274 @@
+/*
+ * 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.integration;
+
+import org.apache.kafka.common.serialization.Serdes;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.KafkaStreams;
+import org.apache.kafka.streams.KeyValue;
+import org.apache.kafka.streams.StreamsBuilder;
+import org.apache.kafka.streams.StreamsConfig;
+import org.apache.kafka.streams.Topology;
+import org.apache.kafka.streams.errors.StreamsException;
+import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster;
+import org.apache.kafka.streams.integration.utils.IntegrationTestUtils;
+import org.apache.kafka.streams.kstream.KStream;
+import org.apache.kafka.streams.kstream.Named;
+import org.apache.kafka.streams.processor.AbstractProcessor;
+import org.apache.kafka.test.IntegrationTest;
+import org.apache.kafka.test.TestUtils;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.ClassRule;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.junit.rules.TestName;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Properties;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.kafka.common.utils.Utils.mkEntry;
+import static org.apache.kafka.common.utils.Utils.mkMap;
+import static org.apache.kafka.common.utils.Utils.mkObjectProperties;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_APPLICATION;
+import static 
org.apache.kafka.streams.errors.StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse.SHUTDOWN_CLIENT;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.cleanStateBeforeTest;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.purgeLocalStreamsState;
+import static 
org.apache.kafka.streams.integration.utils.IntegrationTestUtils.safeUniqueTestName;
+import static org.hamcrest.CoreMatchers.equalTo;
+import static org.hamcrest.MatcherAssert.assertThat;
+import static org.junit.Assert.fail;
+
+@Category(IntegrationTest.class)
+public class StreamsHandlerIntegrationTest {

Review comment:
   sure





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -997,6 +1061,72 @@ private boolean close(final long timeoutMs) {
 }
 }
 
+private void closeToError() {
+if (!setState(State.ERROR)) {
+// if transition failed, it means it was either in PENDING_SHUTDOWN
+// or NOT_RUNNING already; just check that all threads have been 
stopped

Review comment:
   I don't think we actually need it either way so I will just remove it





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +374,63 @@ 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.
+ * 
+ * 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 StreamsUncaughtExceptionHandler 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 StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse 
handleStreamsUncaughtException(final Throwable e,
+   
  final StreamsUncaughtExceptionHandler 
streamsUncaughtExceptionHandler) {
+final StreamsUncaughtExceptionHandler.StreamThreadExceptionResponse 
action = streamsUncaughtExceptionHandler.handle(e);
+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);

Review comment:
   It might be but I do not think that it is necessary





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-05 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -346,18 +351,21 @@ public void setStateListener(final 
KafkaStreams.StateListener listener) {
  * Set the handler invoked when a {@link 
StreamsConfig#NUM_STREAM_THREADS_CONFIG internal thread} abruptly
  * terminates due to an uncaught exception.
  *
- * @param eh the uncaught exception handler for all internal threads; 
{@code null} deletes the current handler
+ * @param uncaughtExceptionHandler the uncaught exception handler for all 
internal threads; {@code null} deletes the current handler
  * @throws IllegalStateException if this {@code KafkaStreams} instance is 
not in state {@link State#CREATED CREATED}.
+ *
+ * @Deprecated Since 2.7.0. Use {@link 
KafkaStreams#setUncaughtExceptionHandler(StreamsUncaughtExceptionHandler)} 
instead.

Review comment:
   yes





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-04 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/StreamsAssignmentProtocolVersions.java
##
@@ -19,7 +19,7 @@
 public final class StreamsAssignmentProtocolVersions {
 public static final int UNKNOWN = -1;
 public static final int EARLIEST_PROBEABLE_VERSION = 3;
-public static final int LATEST_SUPPORTED_VERSION = 8;
+public static final int LATEST_SUPPORTED_VERSION = 9;

Review comment:
   thanks for the reminder. I think I I under stood the test ad 
incrementing to the next version, as the version is now 9





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-04 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
##
@@ -294,7 +304,10 @@ public static StreamThread create(final 
InternalTopologyBuilder builder,
   final long cacheSizeBytes,
   final StateDirectory stateDirectory,
   final StateRestoreListener 
userStateRestoreListener,
-  final int threadIdx) {
+  final int threadIdx,
+  final ShutdownErrorHook 
shutdownErrorHook,
+  final StreamsUncaughtExceptionHandler 
streamsUncaughtExceptionHandler,
+  final AtomicInteger assignmentErrorCode) 
{

Review comment:
   You are right it seems that it is not necessary 





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-04 Thread GitBox


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



##
File path: 
streams/src/main/java/org/apache/kafka/streams/processor/internals/GlobalStreamThread.java
##
@@ -311,6 +317,22 @@ public void run() {
 "Updating global state failed. You can restart KafkaStreams to 
recover from this error.",
 recoverableException
 );
+} catch (final Exception e) {
+if (this.streamsUncaughtExceptionHandler == null) {
+throw e;
+}
+if (Thread.getDefaultUncaughtExceptionHandler() != null && 
newHandler) {
+log.error("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.");

Review comment:
   I think it is simpler to check in the Stream thread because we don't in 
KafkaStreams if the handlers have been set so we would have to check the stream 
thread a global thread so it would be much easier to just check in the thread. 
I do agree that it should be bumped down to warn through.





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-04 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -997,6 +1064,72 @@ private boolean close(final long timeoutMs) {
 }
 }
 
+private void closeToError() {
+if (!setState(State.ERROR)) {
+// if transition failed, it means it was either in PENDING_SHUTDOWN
+// or NOT_RUNNING already; just check that all threads have been 
stopped
+log.info("Can not close to error from state " + state());

Review comment:
   That works





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




[GitHub] [kafka] wcarlson5 commented on a change in pull request #9487: KAFKA-9331: Add a streams specific uncaught exception handler

2020-11-04 Thread GitBox


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



##
File path: streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java
##
@@ -366,6 +374,66 @@ public void setUncaughtExceptionHandler(final 
Thread.UncaughtExceptionHandler eh
 }
 }
 
+/**
+ * Set the handler invoked when a {@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.
+ * 
+ * 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; 
{@code null} deletes the current handler
+ * @throws IllegalStateException if this {@code KafkaStreams} instance is 
not in state {@link State#CREATED CREATED}.
+ * @throws NullPointerException @NotNull if 
streamsUncaughtExceptionHandler is null.

Review comment:
   I don't remember putting it there so it was probably a mistake





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