-
Notifications
You must be signed in to change notification settings - Fork 15.1k
KAFKA-16903: Consider produce error of different task #16222
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Merged
mjsax
merged 1 commit into
apache:trunk
from
cadonna:fix_bug_using_producer_in_aborted_state
Jun 6, 2024
Merged
Changes from all commits
Commits
File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -74,6 +74,7 @@ | |
| import java.util.UUID; | ||
| import java.util.concurrent.Future; | ||
| import java.util.concurrent.atomic.AtomicBoolean; | ||
| import java.util.concurrent.atomic.AtomicReference; | ||
| import java.util.stream.Collectors; | ||
| import java.util.stream.IntStream; | ||
|
|
||
|
|
@@ -754,7 +755,7 @@ public void shouldForwardFlushToStreamsProducer() { | |
| final StreamsProducer streamsProducer = mock(StreamsProducer.class); | ||
| when(streamsProducer.eosEnabled()).thenReturn(false); | ||
| doNothing().when(streamsProducer).flush(); | ||
|
|
||
| when(streamsProducer.sendException()).thenReturn(new AtomicReference<>(null)); | ||
|
Member
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Needed to add this stub to a couple of tests so that they do not throw a |
||
| final ProcessorTopology topology = mock(ProcessorTopology.class); | ||
| when(topology.sinkTopics()).thenReturn(Collections.emptySet()); | ||
|
|
||
|
|
@@ -774,6 +775,7 @@ public void shouldForwardFlushToStreamsProducer() { | |
| public void shouldForwardFlushToStreamsProducerEosEnabled() { | ||
| final StreamsProducer streamsProducer = mock(StreamsProducer.class); | ||
| when(streamsProducer.eosEnabled()).thenReturn(true); | ||
| when(streamsProducer.sendException()).thenReturn(new AtomicReference<>(null)); | ||
| doNothing().when(streamsProducer).flush(); | ||
| final ProcessorTopology topology = mock(ProcessorTopology.class); | ||
|
|
||
|
|
@@ -802,6 +804,7 @@ public void shouldClearOffsetsOnCloseDirty() { | |
| private void shouldClearOffsetsOnClose(final boolean clean) { | ||
| final StreamsProducer streamsProducer = mock(StreamsProducer.class); | ||
| when(streamsProducer.eosEnabled()).thenReturn(true); | ||
| when(streamsProducer.sendException()).thenReturn(new AtomicReference<>(null)); | ||
| final long offset = 1234L; | ||
| final RecordMetadata metadata = new RecordMetadata( | ||
| new TopicPartition(topic, 0), | ||
|
|
@@ -853,7 +856,7 @@ private void shouldClearOffsetsOnClose(final boolean clean) { | |
| public void shouldNotAbortTxOnCloseCleanIfEosEnabled() { | ||
| final StreamsProducer streamsProducer = mock(StreamsProducer.class); | ||
| when(streamsProducer.eosEnabled()).thenReturn(true); | ||
|
|
||
| when(streamsProducer.sendException()).thenReturn(new AtomicReference<>(null)); | ||
| final ProcessorTopology topology = mock(ProcessorTopology.class); | ||
|
|
||
| final RecordCollector collector = new RecordCollectorImpl( | ||
|
|
@@ -872,8 +875,8 @@ public void shouldNotAbortTxOnCloseCleanIfEosEnabled() { | |
| public void shouldAbortTxOnCloseDirtyIfEosEnabled() { | ||
| final StreamsProducer streamsProducer = mock(StreamsProducer.class); | ||
| when(streamsProducer.eosEnabled()).thenReturn(true); | ||
| when(streamsProducer.sendException()).thenReturn(new AtomicReference<>(null)); | ||
| doNothing().when(streamsProducer).abortTransaction(); | ||
|
|
||
| final ProcessorTopology topology = mock(ProcessorTopology.class); | ||
|
|
||
| final RecordCollector collector = new RecordCollectorImpl( | ||
|
|
@@ -1514,6 +1517,64 @@ public void shouldNotCallProductionExceptionHandlerOnClassCastException() { | |
| } | ||
| } | ||
|
|
||
| @Test | ||
| public void shouldNotSendIfSendOfOtherTaskFailedInCallback() { | ||
| final TaskId taskId1 = new TaskId(0, 0); | ||
| final TaskId taskId2 = new TaskId(0, 1); | ||
| final StreamsProducer streamsProducer = mock(StreamsProducer.class); | ||
| when(streamsProducer.eosEnabled()).thenReturn(true); | ||
| when(streamsProducer.sendException()).thenReturn(new AtomicReference<>(null)); | ||
| when(streamsProducer.send(any(), any())).thenAnswer( | ||
| invocation -> { | ||
| final Callback callback = invocation.getArgument(1); | ||
| callback.onCompletion(null, new ProducerFencedException("KABOOM!")); | ||
| return null; | ||
| } | ||
| ); | ||
| final RecordCollector collector1 = new RecordCollectorImpl( | ||
| logContext, | ||
| taskId1, | ||
| streamsProducer, | ||
| productionExceptionHandler, | ||
| streamsMetrics, | ||
| topology | ||
| ); | ||
| collector1.initialize(); | ||
| final RecordCollector collector2 = new RecordCollectorImpl( | ||
| logContext, | ||
| taskId2, | ||
| streamsProducer, | ||
| productionExceptionHandler, | ||
| streamsMetrics, | ||
| topology | ||
| ); | ||
| collector2.initialize(); | ||
| collector1.send( | ||
| topic, | ||
| "key", | ||
| "val", | ||
| null, | ||
| 0, | ||
| null, | ||
| stringSerializer, | ||
| stringSerializer, | ||
| sinkNodeName, | ||
| context | ||
| ); | ||
| assertThrows(StreamsException.class, () -> collector2.send( | ||
| topic, | ||
| "key", | ||
| "val", | ||
| null, | ||
| 1, | ||
| null, | ||
| stringSerializer, | ||
| stringSerializer, | ||
| sinkNodeName, | ||
| context | ||
| )); | ||
| } | ||
|
|
||
| private RecordCollector newRecordCollector(final ProductionExceptionHandler productionExceptionHandler) { | ||
| return new RecordCollectorImpl( | ||
| logContext, | ||
|
|
||
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That is basically the fix.
Notice that now an exception caused by one task can be thrown by a different task. For example:
Task
0_0throws error caused by1_1.