You are viewing a plain text version of this content. The canonical link for it is here.
Posted to jira@kafka.apache.org by "Frederic Arno (JIRA)" <ji...@apache.org> on 2018/04/05 10:40:00 UTC
[jira] [Updated] (KAFKA-6749) TopologyTestDriver fails when topoloy
under test uses EXACTLY_ONCE
[ https://issues.apache.org/jira/browse/KAFKA-6749?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
Frederic Arno updated KAFKA-6749:
---------------------------------
Description:
Stream processing topologies which are configured to use {{EXACTLY_ONCE}} processing guarantee cannot be tested with the {{TopologyTestDriver}}. Tests usually crash with {{java.lang.IllegalStateException: MockProducer hasn't been initialized for transactions}} within the second call to {{TopologyTestDriver.pipeInput()}}, the first call works fine.
Changing the processing guarantee to {{AT_LEAST_ONCE}} makes tests pass.
This is a problem because it is expected that proper processor topologies can be successfully tested using {{TopologyTestDriver}}, however {{TopologyTestDriver}} can't handle {{EXACTLY_ONCE}} and crashes during tests. To a developer, this usually means that there is something wrong with their processor topologies.
Kafka developpers can reproduce this by adding:
{code:java}
put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE);{code}
to line 88 of TopologyTestDriverTest: streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
Originally [reported on the ML|http://mail-archives.apache.org/mod_mbox/kafka-users/201804.mbox/%3C54ab29ad-44e1-35bd-9c16-c1d8d68a88db%40gmail.com%3E].
was:
Stream processing topologies which are configured to use `EXACTLY_ONCE` processing guarantee cannot be tested with the `TopologyTestDriver`. Tests usually crash with `java.lang.IllegalStateException: MockProducer hasn't been initialized for transactions` within the second call to `TopologyTestDriver.pipeInput()`, the first call works fine.
Changing the processing guarantee to `AT_LEAST_ONCE` makes tests pass.
This is a problem because it is expected that proper processor topologies can be successfully tested using `TopologyTestDriver`, however `TopologyTestDriver` can't handle `EXACTLY_ONCE` and crashes during tests. To a developer, this usually means that there is something wrong with their processor topologies.
Kafka developpers can reproduce this by adding:
{code:java}
put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE);{code}
to line 88 of TopologyTestDriverTest: streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
Originally reported here: [http://mail-archives.apache.org/mod_mbox/kafka-users/201804.mbox/%3C54ab29ad-44e1-35bd-9c16-c1d8d68a88db%40gmail.com%3E]
> TopologyTestDriver fails when topoloy under test uses EXACTLY_ONCE
> ------------------------------------------------------------------
>
> Key: KAFKA-6749
> URL: https://issues.apache.org/jira/browse/KAFKA-6749
> Project: Kafka
> Issue Type: Bug
> Components: streams
> Affects Versions: 1.1.0
> Reporter: Frederic Arno
> Priority: Major
>
> Stream processing topologies which are configured to use {{EXACTLY_ONCE}} processing guarantee cannot be tested with the {{TopologyTestDriver}}. Tests usually crash with {{java.lang.IllegalStateException: MockProducer hasn't been initialized for transactions}} within the second call to {{TopologyTestDriver.pipeInput()}}, the first call works fine.
> Changing the processing guarantee to {{AT_LEAST_ONCE}} makes tests pass.
> This is a problem because it is expected that proper processor topologies can be successfully tested using {{TopologyTestDriver}}, however {{TopologyTestDriver}} can't handle {{EXACTLY_ONCE}} and crashes during tests. To a developer, this usually means that there is something wrong with their processor topologies.
> Kafka developpers can reproduce this by adding:
> {code:java}
> put(StreamsConfig.PROCESSING_GUARANTEE_CONFIG, StreamsConfig.EXACTLY_ONCE);{code}
> to line 88 of TopologyTestDriverTest: streams/test-utils/src/test/java/org/apache/kafka/streams/TopologyTestDriverTest.java
> Originally [reported on the ML|http://mail-archives.apache.org/mod_mbox/kafka-users/201804.mbox/%3C54ab29ad-44e1-35bd-9c16-c1d8d68a88db%40gmail.com%3E].
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)