This is an automated email from the ASF dual-hosted git repository.
bbejeck pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/kafka.git
The following commit(s) were added to refs/heads/trunk by this push:
new 11a8a8d KAFKA-8290: Close producer for zombie task (#6636)
11a8a8d is described below
commit 11a8a8d274a16ee2aa8271efc1f7418499372e31
Author: Bill Bejeck <[email protected]>
AuthorDate: Mon May 20 09:02:25 2019 -0400
KAFKA-8290: Close producer for zombie task (#6636)
When we close a task and EOS is enabled we should always close the producer
regardless if the task is in a zombie state (the broker fenced the producer) or
not.
I've added tests that fail without this change.
Reviewers: Matthias J. Sax <[email protected]>, Jason Gustafson
<[email protected]>
---
.../streams/processor/internals/StreamTask.java | 2 ++
.../processor/internals/StreamTaskTest.java | 35 +++++++++++++++++++++-
2 files changed, 36 insertions(+), 1 deletion(-)
diff --git
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
index 3d974a5..80653c5 100644
---
a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
+++
b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
@@ -635,7 +635,9 @@ public class StreamTask extends AbstractTask implements
ProcessorNodePunctuator
// can be ignored: transaction got already aborted by
brokers/transactional-coordinator if this happens
}
+ }
+ if (eosEnabled) {
try {
recordCollector.close();
} catch (final Throwable e) {
diff --git
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
index d22bb1b..ccd94de 100644
---
a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
+++
b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
@@ -1092,6 +1092,19 @@ public class StreamTaskTest {
}
@Test
+ public void shouldOnlyCloseFencedProducerOnUncleanClosedWithEosEnabled() {
+ task = createStatelessTask(createConfig(true));
+ task.initializeTopology();
+ producer.fenceProducer();
+
+ task.close(false, true);
+ task = null;
+
+ assertFalse(producer.transactionAborted());
+ assertTrue(producer.closed());
+ }
+
+ @Test
public void
shouldAbortTransactionButNotCloseProducerIfFencedOnCloseDuringUncleanCloseWithEosEnabled()
{
task = createStatelessTask(createConfig(true));
task.initializeTopology();
@@ -1146,7 +1159,7 @@ public class StreamTaskTest {
public void shouldNotThrowOnCloseIfTaskWasNotInitializedWithEosEnabled() {
task = createStatelessTask(createConfig(true));
- assertTrue(!producer.transactionInFlight());
+ assertFalse(producer.transactionInFlight());
task.close(false, false);
}
@@ -1304,6 +1317,26 @@ public class StreamTaskTest {
}
@Test
+ public void shouldCloseProducerOnUncleanCloseNotZombieWhenEosEnabled() {
+ task = createStatelessTask(createConfig(true));
+ task.initializeTopology();
+ task.close(false, false);
+ task = null;
+
+ assertTrue(producer.closed());
+ }
+
+ @Test
+ public void shouldCloseProducerOnUncleanCloseIsZombieWhenEosEnabled() {
+ task = createStatelessTask(createConfig(true));
+ task.initializeTopology();
+ task.close(false, true);
+ task = null;
+
+ assertTrue(producer.closed());
+ }
+
+ @Test
public void shouldNotViolateAtLeastOnceWhenExceptionOccursDuringFlushing()
{
task = createTaskThatThrowsException(false);
task.initializeStateStores();