cadonna commented on code in PR #17711:
URL: https://github.com/apache/kafka/pull/17711#discussion_r1842176035
##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java:
##########
@@ -538,13 +539,18 @@ public void flush() {
} catch (final RuntimeException exception) {
if (firstException == null) {
// do NOT wrap the error if it is actually caused by
Streams itself
- if (exception instanceof StreamsException)
+ // In case of FailedProcessingException Do not keep
the failed processing exception in the stack trace
+ if (exception instanceof FailedProcessingException)
+ firstException = new
ProcessorStateException(exception.getCause());
Review Comment:
IMO, we should keep it exactly the same as before the
`FailedProcessingException` was introduced:
```suggestion
firstException = new ProcessorStateException(
format("%sFailed to flush state store %s",
logPrefix, store.name()), exception.getCause());
```
##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java:
##########
@@ -771,6 +772,38 @@ public void close() {
assertEquals(exception, thrown);
}
+ @Test
+ public void
shouldThrowProcessorStateExceptionOnFlushIfStoreThrowsAFailedProcessingException()
{
+ final RuntimeException exception = new RuntimeException("KABOOM!");
+ final ProcessorStateManager stateManager =
getStateManager(Task.TaskType.ACTIVE);
+ final MockKeyValueStore stateStore = new
MockKeyValueStore(persistentStoreName, true) {
+ @Override
+ public void flush() {
+ throw new FailedProcessingException("processor", exception);
+ }
+ };
+ stateManager.registerStore(stateStore,
stateStore.stateRestoreCallback, null);
+
+ final ProcessorStateException thrown =
assertThrows(ProcessorStateException.class, stateManager::flush);
+ assertEquals(exception, thrown.getCause());
+ }
+
+ @Test
+ public void
shouldThrowProcessorStateExceptionOnCloseIfStoreThrowsAFailedProcessingException()
{
+ final RuntimeException exception = new RuntimeException("KABOOM!");
+ final ProcessorStateManager stateManager =
getStateManager(Task.TaskType.ACTIVE);
+ final MockKeyValueStore stateStore = new
MockKeyValueStore(persistentStoreName, true) {
+ @Override
+ public void close() {
+ throw new FailedProcessingException("processor", exception);
+ }
+ };
+ stateManager.registerStore(stateStore,
stateStore.stateRestoreCallback, null);
+
+ final ProcessorStateException thrown =
assertThrows(ProcessorStateException.class, stateManager::close);
+ assertEquals(exception, thrown.getCause());
Review Comment:
Same as above
##########
streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java:
##########
@@ -771,6 +772,38 @@ public void close() {
assertEquals(exception, thrown);
}
+ @Test
+ public void
shouldThrowProcessorStateExceptionOnFlushIfStoreThrowsAFailedProcessingException()
{
+ final RuntimeException exception = new RuntimeException("KABOOM!");
+ final ProcessorStateManager stateManager =
getStateManager(Task.TaskType.ACTIVE);
+ final MockKeyValueStore stateStore = new
MockKeyValueStore(persistentStoreName, true) {
+ @Override
+ public void flush() {
+ throw new FailedProcessingException("processor", exception);
+ }
+ };
+ stateManager.registerStore(stateStore,
stateStore.stateRestoreCallback, null);
+
+ final ProcessorStateException thrown =
assertThrows(ProcessorStateException.class, stateManager::flush);
+ assertEquals(exception, thrown.getCause());
Review Comment:
Please also verify the message and that the stack trace does not contain the
`FailedProcessingException`.
##########
streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java:
##########
@@ -538,13 +539,16 @@ public void flush() {
} catch (final RuntimeException exception) {
if (firstException == null) {
// do NOT wrap the error if it is actually caused by
Streams itself
- if (exception instanceof StreamsException)
+ // In case of FailedProcessingException Do not keep
the failed processing exception in the stack trace
+ if (exception instanceof FailedProcessingException)
Review Comment:
Why could you not also add a test for `flushCache()` to the unit tests?
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]