Repository: incubator-samza Updated Branches: refs/heads/0.7.0 eb07ed490 -> b715cf0cc
SAMZA-224. Fix race condition in TestStatefulTask Project: http://git-wip-us.apache.org/repos/asf/incubator-samza/repo Commit: http://git-wip-us.apache.org/repos/asf/incubator-samza/commit/b715cf0c Tree: http://git-wip-us.apache.org/repos/asf/incubator-samza/tree/b715cf0c Diff: http://git-wip-us.apache.org/repos/asf/incubator-samza/diff/b715cf0c Branch: refs/heads/0.7.0 Commit: b715cf0ccf9478ef9ff8fca5bd1b150e04609593 Parents: eb07ed4 Author: Martin Kleppmann <[email protected]> Authored: Thu Jun 5 20:42:31 2014 +0100 Committer: Martin Kleppmann <[email protected]> Committed: Thu Jun 5 21:26:59 2014 +0100 ---------------------------------------------------------------------- .../org/apache/samza/test/integration/TestStatefulTask.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/incubator-samza/blob/b715cf0c/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala ---------------------------------------------------------------------- diff --git a/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala b/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala index 10502a9..dc44a99 100644 --- a/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala +++ b/samza-test/src/test/scala/org/apache/samza/test/integration/TestStatefulTask.scala @@ -214,13 +214,13 @@ class TestStatefulTask { "stores.mystore.msg.serde" -> "string", "stores.mystore.changelog" -> "kafka.mystore", - // Use smallest reset for input streams, so we can fix SAMZA-166. "systems.kafka.samza.factory" -> "org.apache.samza.system.kafka.KafkaSystemFactory", - "systems.kafka.samza.offset.default" -> "oldest", + // Always start consuming at offset 0. This avoids a race condition between + // the producer and the consumer in this test (SAMZA-166, SAMZA-224). + "systems.kafka.samza.offset.default" -> "oldest", // applies to a nonempty topic + "systems.kafka.consumer.auto.offset.reset" -> "smallest", // applies to an empty topic "systems.kafka.samza.msg.serde" -> "string", "systems.kafka.consumer.zookeeper.connect" -> zkConnect, - // Use largest offset for reset, so we can test SAMZA-142. - "systems.kafka.consumer.auto.offset.reset" -> "largest", "systems.kafka.producer.metadata.broker.list" -> ("localhost:%s" format port1)) @Test
