From c1e00d0c5717a23cdf7252e48dcd6e49dd0ad810 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 29 Nov 2017 17:01:17 -0800 Subject: [PATCH 01/16] ingested stream stream join --- .../kstream/internals/KStreamKStreamJoin.java | 9 +- .../integration/JoinIntegrationTest.java | 153 ++++++- .../KStreamRepartitionJoinTest.java | 386 ------------------ .../apache/kafka/test/MockKeyValueMapper.java | 13 + 4 files changed, 170 insertions(+), 391 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 4f26767fe408c..7b71c413f130c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -80,10 +80,17 @@ public void process(final K key, final V1 value) { try (WindowStoreIterator iter = otherWindow.fetch(key, timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; - context().forward(key, joiner.apply(value, iter.next().value)); + + V2 value2 = iter.next().value; + + System.out.println("JOIN " + value + " WITH " + value2); + + context().forward(key, joiner.apply(value, value2)); } if (needOuterJoin) { + System.out.println("JOIN " + value + " WITH null"); + context().forward(key, joiner.apply(value, null)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java index faa581b175c17..82525e1153955 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java @@ -34,15 +34,22 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; +import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import java.util.ArrayList; import java.util.Arrays; +import java.util.Collection; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -55,10 +62,22 @@ * Tests all available joins of Kafka Streams DSL. */ @Category({IntegrationTest.class}) +@RunWith(value = Parameterized.class) public class JoinIntegrationTest { @ClassRule public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory()); + + @Parameterized.Parameters + public static Collection data() { + List values = new ArrayList<>(); + for (boolean cacheEnabled : Arrays.asList(true, false)) + values.add(new Object[] {cacheEnabled}); + return values; + } + private static final String APP_ID = "join-integration-test"; private static final String INPUT_TOPIC_1 = "inputTopicLeft"; private static final String INPUT_TOPIC_2 = "inputTopicRight"; @@ -74,6 +93,8 @@ public class JoinIntegrationTest { private KTable leftTable; private KTable rightTable; + private final boolean cacheEnabled; + private final List> input = Arrays.asList( new Input<>(INPUT_TOPIC_1, (String) null), new Input<>(INPUT_TOPIC_2, (String) null), @@ -99,6 +120,10 @@ public String apply(final String value1, final String value2) { } }; + public JoinIntegrationTest(boolean cacheEnabled) { + this.cacheEnabled = cacheEnabled; + } + @BeforeClass public static void setupConfigsAndUtils() { PRODUCER_CONFIG.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); @@ -113,19 +138,21 @@ public static void setupConfigsAndUtils() { RESULT_CONSUMER_CONFIG.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); + STREAMS_CONFIG.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); + STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); - STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); - STREAMS_CONFIG.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); } @Before public void prepareTopology() throws InterruptedException { CLUSTER.createTopics(INPUT_TOPIC_1, INPUT_TOPIC_2, OUTPUT_TOPIC); + if (!cacheEnabled) + STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + + STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath()); builder = new StreamsBuilder(); leftTable = builder.table(INPUT_TOPIC_1); @@ -197,6 +224,64 @@ public void testInnerKStreamKStream() throws Exception { runTest(expectedResult); } + @Test + public void testMultiInnerKStreamKStream() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-multi-inner-KStream-KStream"); + + final List> expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("A-b-a", "B-b-a", "A-a-b", "B-a-b", "A-b-b", "B-b-b"), + null, + null, + Arrays.asList("C-a-a", "C-b-b"), + Arrays.asList("A-c-c", "B-c-c", "C-c-c"), + null, + null, + null, + Arrays.asList("A-d-d", "B-d-d", "C-d-d"), + Arrays.asList("D-a-a", "D-b-b", "D-c-c", "D-d-d") + ); + + leftStream.join(rightStream, valueJoiner, JoinWindows.of(10000)) + .join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + + @Test + public void testInnerRepartitionedKStreamKStream() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-repartitioned-KStream-KStream"); + + final List> expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Arrays.asList("A-b", "B-b"), + null, + null, + Arrays.asList("C-a", "C-b"), + Arrays.asList("A-c", "B-c", "C-c"), + null, + null, + null, + Arrays.asList("A-d", "B-d", "C-d"), + Arrays.asList("D-a", "D-b", "D-c", "D-d") + ); + + leftStream.map(MockKeyValueMapper.NoOpKeyValueMapper()) + .join(rightStream.flatMap(MockKeyValueMapper.NoOpFlatKeyValueMapper()) + .selectKey(MockKeyValueMapper.SelectKeyKeyValueMapper()), + valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + @Test public void testLeftKStreamKStream() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-KStream-KStream"); @@ -224,6 +309,36 @@ public void testLeftKStreamKStream() throws Exception { runTest(expectedResult); } + @Test + public void testLeftRepartitionedKStreamKStream() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-repartitioned-KStream-KStream"); + + final List> expectedResult = Arrays.asList( + null, + null, + Collections.singletonList("A-null"), + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Arrays.asList("A-b", "B-b"), + null, + null, + Arrays.asList("C-a", "C-b"), + Arrays.asList("A-c", "B-c", "C-c"), + null, + null, + null, + Arrays.asList("A-d", "B-d", "C-d"), + Arrays.asList("D-a", "D-b", "D-c", "D-d") + ); + + leftStream.map(MockKeyValueMapper.NoOpKeyValueMapper()) + .leftJoin(rightStream.flatMap(MockKeyValueMapper.NoOpFlatKeyValueMapper()) + .selectKey(MockKeyValueMapper.SelectKeyKeyValueMapper()), + valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + @Test public void testOuterKStreamKStream() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer-KStream-KStream"); @@ -251,6 +366,36 @@ public void testOuterKStreamKStream() throws Exception { runTest(expectedResult); } + @Test + public void testOuterRepartitionedKStreamKStream() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer-repartitioned-KStream-KStream"); + + final List> expectedResult = Arrays.asList( + null, + null, + Collections.singletonList("A-null"), + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Arrays.asList("A-b", "B-b"), + null, + null, + Arrays.asList("C-a", "C-b"), + Arrays.asList("A-c", "B-c", "C-c"), + null, + null, + null, + Arrays.asList("A-d", "B-d", "C-d"), + Arrays.asList("D-a", "D-b", "D-c", "D-d") + ); + + leftStream.map(MockKeyValueMapper.NoOpKeyValueMapper()) + .outerJoin(rightStream.flatMap(MockKeyValueMapper.NoOpFlatKeyValueMapper()) + .selectKey(MockKeyValueMapper.SelectKeyKeyValueMapper()), + valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + @Test public void testInnerKStreamKTable() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-KStream-KTable"); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java deleted file mode 100644 index 5f6ff449b877f..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KStreamRepartitionJoinTest.java +++ /dev/null @@ -1,386 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.integration; - -import kafka.utils.MockTime; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.Consumed; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; -import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; -import org.apache.kafka.streams.kstream.JoinWindows; -import org.apache.kafka.streams.kstream.Joined; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.MockKeyValueMapper; -import org.apache.kafka.test.MockValueJoiner; -import org.apache.kafka.test.TestUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.TimeUnit; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.Is.is; - -@Category({IntegrationTest.class}) -public class KStreamRepartitionJoinTest { - - private static final int NUM_BROKERS = 1; - private static final long COMMIT_INTERVAL_MS = 300L; - - @ClassRule - public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); - public static final ValueJoiner TOSTRING_JOINER = MockValueJoiner.instance(":"); - private final MockTime mockTime = CLUSTER.time; - private static final long WINDOW_SIZE = TimeUnit.MILLISECONDS.convert(1, TimeUnit.DAYS); - - private StreamsBuilder builder; - private Properties streamsConfiguration; - private KStream streamOne; - private KStream streamTwo; - private KStream streamFour; - private KeyValueMapper> keyMapper; - - private final List - expectedStreamOneTwoJoin = Arrays.asList("1:A", "2:B", "3:C", "4:D", "5:E"); - private KafkaStreams kafkaStreams; - private String streamOneInput; - private String streamTwoInput; - private String streamFourInput; - private static volatile int testNo = 0; - - @Before - public void before() throws InterruptedException { - testNo++; - String applicationId = "kstream-repartition-join-test-" + testNo; - builder = new StreamsBuilder(); - createTopics(); - streamsConfiguration = new Properties(); - streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); - streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL_MS); - streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfiguration.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); - streamsConfiguration.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); - - streamOne = builder.stream(streamOneInput, Consumed.with(Serdes.Long(), Serdes.Integer())); - streamTwo = builder.stream(streamTwoInput, Consumed.with(Serdes.Integer(), Serdes.String())); - streamFour = builder.stream(streamFourInput, Consumed.with(Serdes.Integer(), Serdes.String())); - - keyMapper = MockKeyValueMapper.SelectValueKeyValueMapper(); - } - - @After - public void whenShuttingDown() throws IOException { - if (kafkaStreams != null) { - kafkaStreams.close(); - } - IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); - } - - @Test - public void shouldCorrectlyRepartitionOnJoinOperationsWithZeroSizedCache() throws Exception { - verifyRepartitionOnJoinOperations(0); - } - - @Test - public void shouldCorrectlyRepartitionOnJoinOperationsWithNonZeroSizedCache() throws Exception { - verifyRepartitionOnJoinOperations(10 * 1024 * 1024); - } - - private void verifyRepartitionOnJoinOperations(final int cacheSizeBytes) throws Exception { - streamsConfiguration.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, cacheSizeBytes); - produceMessages(); - final ExpectedOutputOnTopic mapOne = mapStreamOneAndJoin(); - final ExpectedOutputOnTopic mapBoth = mapBothStreamsAndJoin(); - final ExpectedOutputOnTopic mapMapJoin = mapMapJoin(); - final ExpectedOutputOnTopic selectKeyJoin = selectKeyAndJoin(); - final ExpectedOutputOnTopic flatMapJoin = flatMapJoin(); - final ExpectedOutputOnTopic mapRhs = joinMappedRhsStream(); - final ExpectedOutputOnTopic mapJoinJoin = joinTwoMappedStreamsOneThatHasBeenPreviouslyJoined(); - final ExpectedOutputOnTopic leftJoin = mapBothStreamsAndLeftJoin(); - - startStreams(); - - verifyCorrectOutput(mapOne); - verifyCorrectOutput(mapBoth); - verifyCorrectOutput(mapMapJoin); - verifyCorrectOutput(selectKeyJoin); - verifyCorrectOutput(flatMapJoin); - verifyCorrectOutput(mapRhs); - verifyCorrectOutput(mapJoinJoin); - verifyCorrectOutput(leftJoin); - } - - private ExpectedOutputOnTopic mapStreamOneAndJoin() throws InterruptedException { - String mapOneStreamAndJoinOutput = "map-one-join-output-" + testNo; - doJoin(streamOne.map(keyMapper), streamTwo, mapOneStreamAndJoinOutput); - return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, mapOneStreamAndJoinOutput); - } - - private ExpectedOutputOnTopic mapBothStreamsAndJoin() throws InterruptedException { - final KStream map1 = streamOne.map(keyMapper); - final KStream map2 = streamTwo.map(MockKeyValueMapper.NoOpKeyValueMapper()); - - doJoin(map1, map2, "map-both-streams-and-join-" + testNo); - return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, "map-both-streams-and-join-" + testNo); - } - - private ExpectedOutputOnTopic mapMapJoin() throws InterruptedException { - final KStream mapMapStream = streamOne.map( - new KeyValueMapper>() { - @Override - public KeyValue apply(final Long key, final Integer value) { - if (value == null) { - return new KeyValue<>(null, null); - } - return new KeyValue<>(key + value, value); - } - }).map(keyMapper); - - final String outputTopic = "map-map-join-" + testNo; - doJoin(mapMapStream, streamTwo, outputTopic); - return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, outputTopic); - } - - private ExpectedOutputOnTopic selectKeyAndJoin() throws Exception { - - final KStream keySelected = - streamOne.selectKey(MockKeyValueMapper.SelectValueMapper()); - - final String outputTopic = "select-key-join-" + testNo; - doJoin(keySelected, streamTwo, outputTopic); - return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, outputTopic); - } - - private ExpectedOutputOnTopic flatMapJoin() throws InterruptedException { - final KStream flatMapped = streamOne.flatMap( - new KeyValueMapper>>() { - @Override - public Iterable> apply(final Long key, final Integer value) { - return Collections.singletonList(new KeyValue<>(value, value)); - } - }); - - final String outputTopic = "flat-map-join-" + testNo; - doJoin(flatMapped, streamTwo, outputTopic); - - return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, outputTopic); - } - - private ExpectedOutputOnTopic joinMappedRhsStream() throws InterruptedException { - - final String output = "join-rhs-stream-mapped-" + testNo; - CLUSTER.createTopic(output); - streamTwo - .join(streamOne.map(keyMapper), - TOSTRING_JOINER, - getJoinWindow(), - Joined.with(Serdes.Integer(), Serdes.String(), Serdes.Integer())) - .to(Serdes.Integer(), Serdes.String(), output); - - return new ExpectedOutputOnTopic(Arrays.asList("A:1", "B:2", "C:3", "D:4", "E:5"), output); - } - - private ExpectedOutputOnTopic mapBothStreamsAndLeftJoin() throws InterruptedException { - final KStream map1 = streamOne.map(keyMapper); - - final KStream map2 = streamTwo.map(MockKeyValueMapper.NoOpKeyValueMapper()); - - - final String outputTopic = "left-join-" + testNo; - CLUSTER.createTopic(outputTopic); - map1.leftJoin(map2, - TOSTRING_JOINER, - getJoinWindow(), - Joined.with(Serdes.Integer(), Serdes.Integer(), Serdes.String())) - .filterNot(new Predicate() { - @Override - public boolean test(Integer key, String value) { - // filter not left-only join results - return value.substring(2).equals("null"); - } - }) - .to(Serdes.Integer(), Serdes.String(), outputTopic); - - return new ExpectedOutputOnTopic(expectedStreamOneTwoJoin, outputTopic); - } - - private ExpectedOutputOnTopic joinTwoMappedStreamsOneThatHasBeenPreviouslyJoined() throws InterruptedException { - final KStream map1 = streamOne.map(keyMapper); - - final KeyValueMapper> - kvMapper = MockKeyValueMapper.NoOpKeyValueMapper(); - - final KStream map2 = streamTwo.map(kvMapper); - - final KStream join = map1.join(map2, - TOSTRING_JOINER, - getJoinWindow(), - Joined.with(Serdes.Integer(), Serdes.Integer(), Serdes.String())); - - final String topic = "map-join-join-" + testNo; - CLUSTER.createTopic(topic); - join.map(kvMapper) - .join(streamFour.map(kvMapper), - TOSTRING_JOINER, - getJoinWindow(), - Joined.with(Serdes.Integer(), Serdes.String(), Serdes.String())) - .to(Serdes.Integer(), Serdes.String(), topic); - - - return new ExpectedOutputOnTopic(Arrays.asList("1:A:A", "2:B:B", "3:C:C", "4:D:D", "5:E:E"), topic); - } - - private JoinWindows getJoinWindow() { - return JoinWindows.of(WINDOW_SIZE).until(3 * WINDOW_SIZE); - } - - - private class ExpectedOutputOnTopic { - private final List expectedOutput; - private final String outputTopic; - - ExpectedOutputOnTopic(final List expectedOutput, final String outputTopic) { - this.expectedOutput = expectedOutput; - this.outputTopic = outputTopic; - } - } - - private void verifyCorrectOutput(final ExpectedOutputOnTopic expectedOutputOnTopic) - throws InterruptedException { - assertThat(receiveMessages(new StringDeserializer(), - expectedOutputOnTopic.expectedOutput.size(), - expectedOutputOnTopic.outputTopic), - is(expectedOutputOnTopic.expectedOutput)); - } - - private void produceMessages() throws Exception { - produceToStreamOne(); - produceStreamTwoInputTo(streamTwoInput); - produceStreamTwoInputTo(streamFourInput); - - } - - private void produceStreamTwoInputTo(final String streamTwoInput) throws Exception { - IntegrationTestUtils.produceKeyValuesSynchronously( - streamTwoInput, - Arrays.asList( - new KeyValue<>(1, "A"), - new KeyValue<>(2, "B"), - new KeyValue<>(3, "C"), - new KeyValue<>(4, "D"), - new KeyValue<>(5, "E")), - TestUtils.producerConfig( - CLUSTER.bootstrapServers(), - IntegerSerializer.class, - StringSerializer.class, - new Properties()), - mockTime); - } - - private void produceToStreamOne() throws Exception { - IntegrationTestUtils.produceKeyValuesSynchronously( - streamOneInput, - Arrays.asList( - new KeyValue<>(10L, 1), - new KeyValue<>(5L, 2), - new KeyValue<>(12L, 3), - new KeyValue<>(15L, 4), - new KeyValue<>(20L, 5), - new KeyValue(70L, null)), // nulls should be filtered - TestUtils.producerConfig( - CLUSTER.bootstrapServers(), - LongSerializer.class, - IntegerSerializer.class, - new Properties()), - mockTime); - } - - private void createTopics() throws InterruptedException { - streamOneInput = "stream-one-" + testNo; - streamTwoInput = "stream-two-" + testNo; - streamFourInput = "stream-four-" + testNo; - CLUSTER.createTopic(streamOneInput, 2, 1); - CLUSTER.createTopic(streamTwoInput, 2, 1); - CLUSTER.createTopic(streamFourInput, 2, 1); - } - - - private void startStreams() { - kafkaStreams = new KafkaStreams(builder.build(), streamsConfiguration); - kafkaStreams.start(); - } - - - private List receiveMessages(final Deserializer valueDeserializer, - final int numMessages, final String topic) throws InterruptedException { - - final Properties config = new Properties(); - - config.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - config.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "kstream-test"); - config.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - config.setProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, - IntegerDeserializer.class.getName()); - config.setProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, - valueDeserializer.getClass().getName()); - final List received = IntegrationTestUtils.waitUntilMinValuesRecordsReceived( - config, - topic, - numMessages, - 60 * 1000); - Collections.sort(received); - - return received; - } - - private void doJoin(final KStream lhs, - final KStream rhs, - final String outputTopic) throws InterruptedException { - CLUSTER.createTopic(outputTopic); - lhs.join(rhs, - TOSTRING_JOINER, - getJoinWindow(), - Joined.with(Serdes.Integer(), Serdes.Integer(), Serdes.String())) - .to(Serdes.Integer(), Serdes.String(), outputTopic); - } - -} diff --git a/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java index 2ad24d7592eef..d343a4b02d41d 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java +++ b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java @@ -19,6 +19,8 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.KeyValueMapper; +import java.util.Collections; + public class MockKeyValueMapper { @@ -30,6 +32,13 @@ public KeyValue apply(K key, V value) { } } + private static class NoOpFlatKeyValueMapper implements KeyValueMapper>> { + @Override + public Iterable> apply(K key, V value) { + return Collections.singletonList(KeyValue.pair(key, value)); + } + } + private static class SelectValueKeyValueMapper implements KeyValueMapper> { @Override public KeyValue apply(K key, V value) { @@ -56,6 +65,10 @@ public static KeyValueMapper SelectKeyKeyValueMapper() { } + public static KeyValueMapper>> NoOpFlatKeyValueMapper() { + return new NoOpFlatKeyValueMapper<>(); + } + public static KeyValueMapper> NoOpKeyValueMapper() { return new NoOpKeyValueMapper<>(); } From 4d95bdb65c4878301d775b69a204e6af1102c0e3 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 18 Dec 2017 11:09:06 -0800 Subject: [PATCH 02/16] create a base abstract join class --- .../kstream/internals/KTableKTableJoin.java | 5 + .../integration/JoinIntegrationTest.java | 371 ++++++++---------- 2 files changed, 165 insertions(+), 211 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java index 1b26a5b00dc2e..892aaed010f0a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java @@ -84,7 +84,10 @@ public void process(final K key, final Change change) { R oldValue = null; final V2 value2 = valueGetter.get(key); + if (value2 == null) { + System.out.println("KTABLE JOIN " + change + " WITH " + value2 + ": NO RESULT"); + return; } @@ -96,6 +99,8 @@ public void process(final K key, final Change change) { oldValue = joiner.apply(change.oldValue, value2); } + System.out.println("KTABLE JOIN " + change + " WITH " + value2 + ": RESULT " + new Change<>(newValue, oldValue)); + context().forward(key, new Change<>(newValue, oldValue)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java index 82525e1153955..a2cf39c1f4082 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java @@ -17,7 +17,9 @@ package org.apache.kafka.streams.integration; import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serdes; @@ -63,140 +65,29 @@ */ @Category({IntegrationTest.class}) @RunWith(value = Parameterized.class) -public class JoinIntegrationTest { - @ClassRule - public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); - - @Rule - public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory()); - - @Parameterized.Parameters - public static Collection data() { - List values = new ArrayList<>(); - for (boolean cacheEnabled : Arrays.asList(true, false)) - values.add(new Object[] {cacheEnabled}); - return values; - } - - private static final String APP_ID = "join-integration-test"; - private static final String INPUT_TOPIC_1 = "inputTopicLeft"; - private static final String INPUT_TOPIC_2 = "inputTopicRight"; - private static final String OUTPUT_TOPIC = "outputTopic"; - - private final static Properties PRODUCER_CONFIG = new Properties(); - private final static Properties RESULT_CONSUMER_CONFIG = new Properties(); - private final static Properties STREAMS_CONFIG = new Properties(); - - private StreamsBuilder builder; +public class JoinIntegrationTest extends AbstractJoinIntegrationTest { private KStream leftStream; private KStream rightStream; private KTable leftTable; private KTable rightTable; - private final boolean cacheEnabled; - - private final List> input = Arrays.asList( - new Input<>(INPUT_TOPIC_1, (String) null), - new Input<>(INPUT_TOPIC_2, (String) null), - new Input<>(INPUT_TOPIC_1, "A"), - new Input<>(INPUT_TOPIC_2, "a"), - new Input<>(INPUT_TOPIC_1, "B"), - new Input<>(INPUT_TOPIC_2, "b"), - new Input<>(INPUT_TOPIC_1, (String) null), - new Input<>(INPUT_TOPIC_2, (String) null), - new Input<>(INPUT_TOPIC_1, "C"), - new Input<>(INPUT_TOPIC_2, "c"), - new Input<>(INPUT_TOPIC_2, (String) null), - new Input<>(INPUT_TOPIC_1, (String) null), - new Input<>(INPUT_TOPIC_2, (String) null), - new Input<>(INPUT_TOPIC_2, "d"), - new Input<>(INPUT_TOPIC_1, "D") - ); - - private final ValueJoiner valueJoiner = new ValueJoiner() { - @Override - public String apply(final String value1, final String value2) { - return value1 + "-" + value2; - } - }; - public JoinIntegrationTest(boolean cacheEnabled) { - this.cacheEnabled = cacheEnabled; - } - - @BeforeClass - public static void setupConfigsAndUtils() { - PRODUCER_CONFIG.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - PRODUCER_CONFIG.put(ProducerConfig.ACKS_CONFIG, "all"); - PRODUCER_CONFIG.put(ProducerConfig.RETRIES_CONFIG, 0); - PRODUCER_CONFIG.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class); - PRODUCER_CONFIG.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - - RESULT_CONSUMER_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - RESULT_CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, APP_ID + "-result-consumer"); - RESULT_CONSUMER_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - RESULT_CONSUMER_CONFIG.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); - RESULT_CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - - STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - STREAMS_CONFIG.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); - STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); - STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); - STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); + super(cacheEnabled); } @Before public void prepareTopology() throws InterruptedException { - CLUSTER.createTopics(INPUT_TOPIC_1, INPUT_TOPIC_2, OUTPUT_TOPIC); - if (!cacheEnabled) - STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + super.prepareEnvironment(); - STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath()); + APP_ID = "stream-stream-join-integration-test"; builder = new StreamsBuilder(); - leftTable = builder.table(INPUT_TOPIC_1); - rightTable = builder.table(INPUT_TOPIC_2); + leftTable = builder.table(INPUT_TOPIC_LEFT); + rightTable = builder.table(INPUT_TOPIC_RIGHT); leftStream = leftTable.toStream(); rightStream = rightTable.toStream(); } - @After - public void cleanup() throws InterruptedException { - CLUSTER.deleteTopicsAndWait(120000, INPUT_TOPIC_1, INPUT_TOPIC_2, OUTPUT_TOPIC); - } - - private void checkResult(final String outputTopic, final List expectedResult) throws InterruptedException { - if (expectedResult != null) { - final List result = IntegrationTestUtils.waitUntilMinValuesRecordsReceived(RESULT_CONSUMER_CONFIG, outputTopic, expectedResult.size(), 30 * 1000L); - assertThat(result, is(expectedResult)); - } - } - - /* - * Runs the actual test. Checks the result after each input record to ensure fixed processing order. - * If an input tuple does not trigger any result, "expectedResult" should contain a "null" entry - */ - private void runTest(final List> expectedResult) throws Exception { - assert expectedResult.size() == input.size(); - - IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); - final KafkaStreams streams = new KafkaStreams(builder.build(), STREAMS_CONFIG); - try { - streams.start(); - - long ts = System.currentTimeMillis(); - - final Iterator> resultIterator = expectedResult.iterator(); - for (final Input singleInput : input) { - IntegrationTestUtils.produceKeyValuesSynchronouslyWithTimestamp(singleInput.topic, Collections.singleton(singleInput.record), PRODUCER_CONFIG, ++ts); - checkResult(OUTPUT_TOPIC, resultIterator.next()); - } - } finally { - streams.close(); - } - } - @Test public void testInnerKStreamKStream() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-KStream-KStream"); @@ -224,34 +115,6 @@ public void testInnerKStreamKStream() throws Exception { runTest(expectedResult); } - @Test - public void testMultiInnerKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-multi-inner-KStream-KStream"); - - final List> expectedResult = Arrays.asList( - null, - null, - null, - Collections.singletonList("A-a-a"), - Collections.singletonList("B-a-a"), - Arrays.asList("A-b-a", "B-b-a", "A-a-b", "B-a-b", "A-b-b", "B-b-b"), - null, - null, - Arrays.asList("C-a-a", "C-b-b"), - Arrays.asList("A-c-c", "B-c-c", "C-c-c"), - null, - null, - null, - Arrays.asList("A-d-d", "B-d-d", "C-d-d"), - Arrays.asList("D-a-a", "D-b-b", "D-c-c", "D-d-d") - ); - - leftStream.join(rightStream, valueJoiner, JoinWindows.of(10000)) - .join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); - - runTest(expectedResult); - } - @Test public void testInnerRepartitionedKStreamKStream() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-repartitioned-KStream-KStream"); @@ -396,6 +259,38 @@ public void testOuterRepartitionedKStreamKStream() throws Exception { runTest(expectedResult); } + @Test + public void testMultiInnerKStreamKStream() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-multi-inner-KStream-KStream"); + + final List> expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("A-b-a", "B-b-a", "A-a-b", "B-a-b", "A-b-b", "B-b-b"), + null, + null, + Arrays.asList("C-a-a", "C-a-b", "C-b-a", "C-b-b"), + Arrays.asList("A-c-a", "A-c-b", "B-c-a", "B-c-b", "C-c-a", "C-c-b", "A-a-c", "B-a-c", + "A-b-c", "B-b-c", "C-a-c", "C-b-c", "A-c-c", "B-c-c", "C-c-c"), + null, + null, + null, + Arrays.asList("A-d-a", "A-d-b", "A-d-c", "B-d-a", "B-d-b", "B-d-c", "C-d-a", "C-d-b", "C-d-c", + "A-a-d", "B-a-d", "A-b-d", "B-b-d", "C-a-d", "C-b-d", "A-c-d", "B-c-d", "C-c-d", + "A-d-d", "B-d-d", "C-d-d"), + Arrays.asList("D-a-a", "D-a-b", "D-a-c", "D-a-d", "D-b-a", "D-b-b", "D-b-c", "D-b-d", "D-c-a", + "D-c-b", "D-c-c", "D-c-d", "D-d-a", "D-d-b", "D-d-c", "D-d-d") + ); + + leftStream.join(rightStream, valueJoiner, JoinWindows.of(10000)) + .join(rightStream, valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + @Test public void testInnerKStreamKTable() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-KStream-KTable"); @@ -454,25 +349,47 @@ public void testLeftKStreamKTable() throws Exception { public void testInnerKTableKTable() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-KTable-KTable"); - final List> expectedResult = Arrays.asList( - null, - null, - null, - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList((String) null), - null, - null, - Collections.singletonList("C-c"), - Collections.singletonList((String) null), - null, - null, - null, - Collections.singletonList("D-d") - ); + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = Arrays.asList( + null, + null, + null, + Arrays.asList("A-a", "A-a"), // dup + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + Collections.singletonList((String) null), // dup + Collections.singletonList("C-c"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList((String) null), // dup + Collections.singletonList("D-d") + ); + } else { + expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("C-c"), + Collections.singletonList((String) null), + null, + null, + null, + Collections.singletonList("D-d") + ); + } - leftTable.join(rightTable, valueJoiner).to(OUTPUT_TOPIC); + leftTable.join(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -481,25 +398,47 @@ public void testInnerKTableKTable() throws Exception { public void testLeftKTableKTable() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-KTable-KTable"); - final List> expectedResult = Arrays.asList( - null, - null, - Collections.singletonList("A-null"), - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList((String) null), - null, - Collections.singletonList("C-null"), - Collections.singletonList("C-c"), - Collections.singletonList("C-null"), - Collections.singletonList((String) null), - null, - null, - Collections.singletonList("D-d") - ); + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = Arrays.asList( + null, + null, + Arrays.asList("A-null", "A-null"), // dup + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + Arrays.asList("C-null", "C-null"), // dup + Collections.singletonList("C-c"), + Collections.singletonList("C-null"), + Collections.singletonList((String) null), + null, + null, + Arrays.asList("D-d", "D-d") + ); + } else { + expectedResult = Arrays.asList( + null, + null, + Collections.singletonList("A-null"), + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + Collections.singletonList("C-null"), + Collections.singletonList("C-c"), + Collections.singletonList("C-null"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("D-d") + ); + } - leftTable.leftJoin(rightTable, valueJoiner).to(OUTPUT_TOPIC); + leftTable.leftJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); runTest(expectedResult); } @@ -508,38 +447,48 @@ public void testLeftKTableKTable() throws Exception { public void testOuterKTableKTable() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer-KTable-KTable"); - final List> expectedResult = Arrays.asList( - null, - null, - Collections.singletonList("A-null"), - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList("null-b"), - Collections.singletonList((String) null), - Collections.singletonList("C-null"), - Collections.singletonList("C-c"), - Collections.singletonList("C-null"), - Collections.singletonList((String) null), - null, - Collections.singletonList("null-d"), - Collections.singletonList("D-d") - ); + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = Arrays.asList( + null, + null, + Arrays.asList("A-null", "A-null"), + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList("null-b"), + Collections.singletonList((String) null), + Collections.singletonList("C-null"), + Collections.singletonList("C-c"), + Collections.singletonList("C-null"), + Collections.singletonList((String) null), + null, + Collections.singletonList("null-d"), + Collections.singletonList("D-d") + ); + } else { + expectedResult = Arrays.asList( + null, + null, + Collections.singletonList("A-null"), + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList("null-b"), + Collections.singletonList((String) null), + Collections.singletonList("C-null"), + Collections.singletonList("C-c"), + Collections.singletonList("C-null"), + Collections.singletonList((String) null), + null, + Collections.singletonList("null-d"), + Collections.singletonList("D-d") + ); + } - leftTable.outerJoin(rightTable, valueJoiner).to(OUTPUT_TOPIC); + leftTable.outerJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); runTest(expectedResult); } - - private final class Input { - String topic; - KeyValue record; - - private final long anyUniqueKey = 0L; - - Input(final String topic, final V value) { - this.topic = topic; - record = KeyValue.pair(anyUniqueKey, value); - } - } } From defd7edb6792e74f411790422ec65bda876486dd Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 19 Dec 2017 12:53:38 -0800 Subject: [PATCH 03/16] Add more tests --- .../AbstractJoinIntegrationTest.java | 222 ++++++++++++ .../TableTableJoinIntegrationTest.java | 332 ++++++++++++++++++ 2 files changed, 554 insertions(+) create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java new file mode 100644 index 0000000000000..bad300b6d9964 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -0,0 +1,222 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.integration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.JoinWindows; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Reducer; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.MockKeyValueMapper; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; + +/** + * Tests all available joins of Kafka Streams DSL. + */ +@Category({IntegrationTest.class}) +@RunWith(value = Parameterized.class) +public abstract class AbstractJoinIntegrationTest { + @ClassRule + public static final EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(1); + + @Rule + public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory()); + + @Parameterized.Parameters + public static Collection data() { + List values = new ArrayList<>(); + for (boolean cacheEnabled : Arrays.asList(true, false)) + values.add(new Object[] {cacheEnabled}); + return values; + } + + static String APP_ID; + + static final String INPUT_TOPIC_LEFT = "inputTopicLeft"; + static final String INPUT_TOPIC_RIGHT = "inputTopicRight"; + static final String OUTPUT_TOPIC = "outputTopic"; + + private final static Properties PRODUCER_CONFIG = new Properties(); + private final static Properties RESULT_CONSUMER_CONFIG = new Properties(); + final static Properties STREAMS_CONFIG = new Properties(); + + private KafkaProducer producer; + + StreamsBuilder builder; + + private final List> input = Arrays.asList( + new Input<>(INPUT_TOPIC_LEFT, (String) null), + new Input<>(INPUT_TOPIC_RIGHT, (String) null), + new Input<>(INPUT_TOPIC_LEFT, "A"), + new Input<>(INPUT_TOPIC_RIGHT, "a"), + new Input<>(INPUT_TOPIC_LEFT, "B"), + new Input<>(INPUT_TOPIC_RIGHT, "b"), + new Input<>(INPUT_TOPIC_LEFT, (String) null), + new Input<>(INPUT_TOPIC_RIGHT, (String) null), + new Input<>(INPUT_TOPIC_LEFT, "C"), + new Input<>(INPUT_TOPIC_RIGHT, "c"), + new Input<>(INPUT_TOPIC_RIGHT, (String) null), + new Input<>(INPUT_TOPIC_LEFT, (String) null), + new Input<>(INPUT_TOPIC_RIGHT, (String) null), + new Input<>(INPUT_TOPIC_RIGHT, "d"), + new Input<>(INPUT_TOPIC_LEFT, "D") + ); + + final ValueJoiner valueJoiner = new ValueJoiner() { + @Override + public String apply(final String value1, final String value2) { + return value1 + "-" + value2; + } + }; + + final Reducer reducer = new Reducer() { + @Override + public String apply(final String value1, final String value2) { + return value1 + "-" + value2; + } + }; + + final boolean cacheEnabled; + + AbstractJoinIntegrationTest(boolean cacheEnabled) { + this.cacheEnabled = cacheEnabled; + } + + @BeforeClass + public static void setupConfigsAndUtils() { + PRODUCER_CONFIG.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + PRODUCER_CONFIG.put(ProducerConfig.ACKS_CONFIG, "all"); + PRODUCER_CONFIG.put(ProducerConfig.RETRIES_CONFIG, 0); + PRODUCER_CONFIG.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class); + PRODUCER_CONFIG.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + + RESULT_CONSUMER_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + RESULT_CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, APP_ID + "-result-consumer"); + RESULT_CONSUMER_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + RESULT_CONSUMER_CONFIG.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); + RESULT_CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + + STREAMS_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + STREAMS_CONFIG.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); + STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); + STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); + STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + // TODO: set commit interval to smaller value after KAFKA-4309 + STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + } + + void prepareEnvironment() throws InterruptedException { + CLUSTER.createTopics(INPUT_TOPIC_LEFT, INPUT_TOPIC_RIGHT, OUTPUT_TOPIC); + + if (!cacheEnabled) + STREAMS_CONFIG.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); + + STREAMS_CONFIG.put(StreamsConfig.STATE_DIR_CONFIG, testFolder.getRoot().getPath()); + + producer = new KafkaProducer<>(PRODUCER_CONFIG); + } + + @After + public void cleanup() throws InterruptedException { + CLUSTER.deleteTopicsAndWait(120000, INPUT_TOPIC_LEFT, INPUT_TOPIC_RIGHT, OUTPUT_TOPIC); + } + + private void checkResult(final String outputTopic, final List expectedResult) throws InterruptedException { + if (expectedResult != null) { + final List result = IntegrationTestUtils.waitUntilMinValuesRecordsReceived(RESULT_CONSUMER_CONFIG, outputTopic, expectedResult.size(), 30 * 1000L); + assertThat(result, is(expectedResult)); + } + } + + /* + * Runs the actual test. Checks the result after each input record to ensure fixed processing order. + * If an input tuple does not trigger any result, "expectedResult" should contain a "null" entry + */ + void runTest(final List> expectedResult) throws Exception { + assert expectedResult.size() == input.size(); + + IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); + final KafkaStreams streams = new KafkaStreams(builder.build(), STREAMS_CONFIG); + + try { + streams.start(); + + long ts = System.currentTimeMillis(); + + final Iterator> resultIterator = expectedResult.iterator(); + for (final Input singleInput : input) { + producer.send(new ProducerRecord<>(singleInput.topic, null, ++ts, singleInput.record.key, singleInput.record.value)).get(); + + List expected = resultIterator.next(); + System.out.println("checking result: " + expected); + + checkResult(OUTPUT_TOPIC, expected); + } + } finally { + streams.close(); + } + } + + private final class Input { + String topic; + KeyValue record; + + private final long anyUniqueKey = 0L; + + Input(final String topic, final V value) { + this.topic = topic; + record = KeyValue.pair(anyUniqueKey, value); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java new file mode 100644 index 0000000000000..4af9d65d8ea08 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -0,0 +1,332 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kafka.streams.integration; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsBuilder; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import org.apache.kafka.streams.kstream.JoinWindows; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.test.IntegrationTest; +import org.apache.kafka.test.MockKeyValueMapper; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Rule; +import org.junit.Test; +import org.junit.experimental.categories.Category; +import org.junit.rules.TemporaryFolder; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Properties; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.core.Is.is; + +/** + * Tests all available joins of Kafka Streams DSL. + */ +@Category({IntegrationTest.class}) +@RunWith(value = Parameterized.class) +public class TableTableJoinIntegrationTest extends AbstractJoinIntegrationTest { + private KTable leftTable; + private KTable rightTable; + + public TableTableJoinIntegrationTest(boolean cacheEnabled) { + super(cacheEnabled); + } + + @Before + public void prepareTopology() throws InterruptedException { + super.prepareEnvironment(); + + APP_ID = "table-table-join-integration-test"; + + builder = new StreamsBuilder(); + leftTable = builder.table(INPUT_TOPIC_LEFT); + rightTable = builder.table(INPUT_TOPIC_RIGHT); + } + + private List> dedupExpectedJoinResult = Arrays.asList( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + Arrays.asList("D-d", "D-d") + ); + + private List> dedupExpectedMultiJoinResult = Arrays.asList( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + Arrays.asList("D-d-d", "D-d-d") + ); + + @Test + public void testInner() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner"); + + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = dedupExpectedJoinResult; + } else { + expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("C-c"), + Collections.singletonList((String) null), + null, + null, + null, + Collections.singletonList("D-d") + ); + } + + leftTable.join(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + + @Test + public void testLeft() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left"); + + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = dedupExpectedJoinResult; + } else { + expectedResult = Arrays.asList( + null, + null, + Collections.singletonList("A-null"), + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + Collections.singletonList("C-null"), + Collections.singletonList("C-c"), + Collections.singletonList("C-null"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("D-d") + ); + } + + leftTable.leftJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + + @Test + public void testOuter() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); + + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = dedupExpectedJoinResult; + } else { + expectedResult = Arrays.asList( + null, + null, + Collections.singletonList("A-null"), + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList("null-b"), + Collections.singletonList((String) null), + Collections.singletonList("C-null"), + Collections.singletonList("C-c"), + Collections.singletonList("C-null"), + Collections.singletonList((String) null), + null, + Collections.singletonList("null-d"), + Collections.singletonList("D-d") + ); + } + + leftTable.outerJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + + @Test + public void testInnerInner() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-inner"); + + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = dedupExpectedMultiJoinResult; + } else { + expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a-a"), + Collections.singletonList("B-a-a"), + Collections.singletonList("B-b-b"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("C-c-c"), + Collections.singletonList((String) null), + null, + null, + null, + Collections.singletonList("D-d-d") + ); + } + + leftTable.join(rightTable, valueJoiner) + .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) + .reduce(reducer, reducer) + .join(rightTable, valueJoiner) + .toStream().to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + + @Test + public void testInnerLeft() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-left"); + + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = dedupExpectedMultiJoinResult; + } else { + expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("C-c"), + Collections.singletonList((String) null), + null, + null, + null, + Collections.singletonList("D-d") + ); + } + + leftTable.join(rightTable, valueJoiner) + .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) + .reduce(reducer, reducer) + .leftJoin(rightTable, valueJoiner) + .toStream().to(OUTPUT_TOPIC); + + runTest(expectedResult); + } + + @Test + public void testInnerOuter() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-outer"); + + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed + List> expectedResult; + if (cacheEnabled) { + expectedResult = dedupExpectedMultiJoinResult; + } else { + expectedResult = Arrays.asList( + null, + null, + null, + Collections.singletonList("A-a"), + Collections.singletonList("B-a"), + Collections.singletonList("B-b"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("C-c"), + Collections.singletonList((String) null), + null, + null, + null, + Collections.singletonList("D-d") + ); + } + + leftTable.join(rightTable, valueJoiner) + .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) + .reduce(reducer, reducer) + .leftJoin(rightTable, valueJoiner) + .toStream().to(OUTPUT_TOPIC); + + runTest(expectedResult); + } +} From 73f58fbcf6c951b992b62f238ba503d2572d782c Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Wed, 20 Dec 2017 15:20:16 -0800 Subject: [PATCH 04/16] consider using mock time --- .../streams/processor/internals/StreamThread.java | 3 +++ .../integration/AbstractJoinIntegrationTest.java | 14 +++++++++----- .../integration/TableTableJoinIntegrationTest.java | 3 +-- 3 files changed, 13 insertions(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 696081d36c7e5..91c5a18c610a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -985,6 +985,9 @@ private long adjustRecordsProcessedBeforeCommit(final long prevRecordsProcessedB * or if the task producer got fenced (EOS) */ void maybeCommit(final long now) { + + System.out.println("\nDECIDING whether to commit: " + commitTimeMs + " + " + lastCommitMs + " v.s. NOW: " + now + "\n"); + if (commitTimeMs >= 0 && lastCommitMs + commitTimeMs < now) { if (log.isTraceEnabled()) { log.trace("Committing all active tasks {} and standby tasks {} since {}ms has elapsed (commit interval is {}ms)", diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index bad300b6d9964..3f44669606f66 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -25,6 +25,8 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; @@ -83,13 +85,15 @@ public static Collection data() { static String APP_ID; - static final String INPUT_TOPIC_LEFT = "inputTopicLeft"; + static final Time TIME = new MockTime(); + static final Long COMMIT_INTERVAL = 100L; + static final Properties STREAMS_CONFIG = new Properties(); static final String INPUT_TOPIC_RIGHT = "inputTopicRight"; + static final String INPUT_TOPIC_LEFT = "inputTopicLeft"; static final String OUTPUT_TOPIC = "outputTopic"; private final static Properties PRODUCER_CONFIG = new Properties(); private final static Properties RESULT_CONSUMER_CONFIG = new Properties(); - final static Properties STREAMS_CONFIG = new Properties(); private KafkaProducer producer; @@ -152,8 +156,7 @@ public static void setupConfigsAndUtils() { STREAMS_CONFIG.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); STREAMS_CONFIG.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Long().getClass()); STREAMS_CONFIG.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); - // TODO: set commit interval to smaller value after KAFKA-4309 - STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 1000); + STREAMS_CONFIG.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, COMMIT_INTERVAL); } void prepareEnvironment() throws InterruptedException { @@ -187,7 +190,7 @@ void runTest(final List> expectedResult) throws Exception { assert expectedResult.size() == input.size(); IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); - final KafkaStreams streams = new KafkaStreams(builder.build(), STREAMS_CONFIG); + final KafkaStreams streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG), TIME); try { streams.start(); @@ -197,6 +200,7 @@ void runTest(final List> expectedResult) throws Exception { final Iterator> resultIterator = expectedResult.iterator(); for (final Input singleInput : input) { producer.send(new ProducerRecord<>(singleInput.topic, null, ++ts, singleInput.record.key, singleInput.record.value)).get(); + TIME.sleep(COMMIT_INTERVAL + 1); List expected = resultIterator.next(); System.out.println("checking result: " + expected); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java index 4af9d65d8ea08..9f8ace22c9846 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -84,6 +84,7 @@ public void prepareTopology() throws InterruptedException { rightTable = builder.table(INPUT_TOPIC_RIGHT); } + // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed private List> dedupExpectedJoinResult = Arrays.asList( null, null, @@ -124,7 +125,6 @@ public void prepareTopology() throws InterruptedException { public void testInner() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner"); - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed List> expectedResult; if (cacheEnabled) { expectedResult = dedupExpectedJoinResult; @@ -157,7 +157,6 @@ public void testInner() throws Exception { public void testLeft() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left"); - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed List> expectedResult; if (cacheEnabled) { expectedResult = dedupExpectedJoinResult; From ae2ba110f071a2a66af73f349c861569525fbc60 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 8 Jan 2018 09:36:51 -0800 Subject: [PATCH 05/16] fix a bug in flushing ordering --- .../kstream/internals/KStreamKStreamJoin.java | 6 - .../internals/KTableKTableLeftJoin.java | 3 + .../internals/KTableKTableOuterJoin.java | 3 + .../kstream/internals/KTableSource.java | 3 + .../KTableSourceValueGetterSupplier.java | 2 + .../processor/internals/StreamThread.java | 3 - .../state/internals/CachingKeyValueStore.java | 5 +- .../AbstractJoinIntegrationTest.java | 66 +++-- .../TableTableJoinIntegrationTest.java | 242 ++++++++---------- 9 files changed, 161 insertions(+), 172 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 7b71c413f130c..9e4f19429d1da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -80,17 +80,11 @@ public void process(final K key, final V1 value) { try (WindowStoreIterator iter = otherWindow.fetch(key, timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; - V2 value2 = iter.next().value; - - System.out.println("JOIN " + value + " WITH " + value2); - context().forward(key, joiner.apply(value, value2)); } if (needOuterJoin) { - System.out.println("JOIN " + value + " WITH null"); - context().forward(key, joiner.apply(value, null)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index c308a0dd36421..2e2f519486de6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -74,6 +74,9 @@ public void process(final K key, final Change change) { R oldValue = null; final V2 value2 = valueGetter.get(key); + + System.out.println("KTABLE LEFT-JOIN: " + change + " v.s. " + value2); + if (value2 == null && change.newValue == null && change.oldValue == null) { return; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 9cee4f313182a..7649c152faa4d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -73,6 +73,9 @@ public void process(final K key, final Change change) { R oldValue = null; final V2 value2 = valueGetter.get(key); + + System.out.println("KTABLE OUTER-JOIN: " + change + " v.s. " + value2); + if (value2 == null && change.newValue == null && change.oldValue == null) { return; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index de8bf2e7bb60f..a404915cf98ae 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -60,6 +60,9 @@ public void process(K key, V value) { if (key == null) return; V oldValue = store.get(key); + + System.out.println("PUT " + key + "<-" + value + " in store " + storeName); + store.put(key, value); tupleForwarder.maybeForward(key, value, oldValue); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java index 714b8c50dfa37..5322e62ad42b2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java @@ -46,6 +46,8 @@ public void init(ProcessorContext context) { } public V get(K key) { + System.out.println("GET " + key + "<-" + store.get(key) + " from store " + storeName); + return store.get(key); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 91c5a18c610a0..696081d36c7e5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -985,9 +985,6 @@ private long adjustRecordsProcessedBeforeCommit(final long prevRecordsProcessedB * or if the task producer got fenced (EOS) */ void maybeCommit(final long now) { - - System.out.println("\nDECIDING whether to commit: " + commitTimeMs + " + " + lastCommitMs + " v.s. NOW: " + now + "\n"); - if (commitTimeMs >= 0 && lastCommitMs + commitTimeMs < now) { if (log.isTraceEnabled()) { log.trace("Committing all active tasks {} and standby tasks {} since {}ms has elapsed (commit interval is {}ms)", diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index f0669a4f6eeb9..2c9ce5ee3bb61 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -87,14 +87,15 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final Intern try { context.setRecordContext(entry.recordContext()); if (flushListener != null) { - final V oldValue = sendOldValues ? serdes.valueFrom(underlying.get(entry.key())) : null; + + underlying.put(entry.key(), entry.newValue()); + flushListener.apply(serdes.keyFrom(entry.key().get()), serdes.valueFrom(entry.newValue()), oldValue); } - underlying.put(entry.key(), entry.newValue()); } finally { context.setRecordContext(current); } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index 3f44669606f66..0cd81ecb98145 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -25,28 +25,20 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; -import org.apache.kafka.streams.kstream.JoinWindows; -import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.MockKeyValueMapper; +import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; import org.junit.After; -import org.junit.Before; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Rule; -import org.junit.Test; import org.junit.experimental.categories.Category; import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; @@ -55,10 +47,10 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.Iterator; import java.util.List; import java.util.Properties; +import java.util.concurrent.atomic.AtomicBoolean; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.core.Is.is; @@ -85,8 +77,7 @@ public static Collection data() { static String APP_ID; - static final Time TIME = new MockTime(); - static final Long COMMIT_INTERVAL = 100L; + private static final Long COMMIT_INTERVAL = 100L; static final Properties STREAMS_CONFIG = new Properties(); static final String INPUT_TOPIC_RIGHT = "inputTopicRight"; static final String INPUT_TOPIC_LEFT = "inputTopicLeft"; @@ -98,6 +89,8 @@ public static Collection data() { private KafkaProducer producer; StreamsBuilder builder; + int numRecordsExpected = 0; + AtomicBoolean finalResultReached = new AtomicBoolean(false); private final List> input = Arrays.asList( new Input<>(INPUT_TOPIC_LEFT, (String) null), @@ -124,13 +117,6 @@ public String apply(final String value1, final String value2) { } }; - final Reducer reducer = new Reducer() { - @Override - public String apply(final String value1, final String value2) { - return value1 + "-" + value2; - } - }; - final boolean cacheEnabled; AbstractJoinIntegrationTest(boolean cacheEnabled) { @@ -182,6 +168,11 @@ private void checkResult(final String outputTopic, final List expectedRe } } + private void checkResult(final String outputTopic, final String expectedFinalResult, final int expectedTotalNumRecords) throws InterruptedException { + final List result = IntegrationTestUtils.waitUntilMinValuesRecordsReceived(RESULT_CONSUMER_CONFIG, outputTopic, expectedTotalNumRecords, 30 * 1000L); + assertThat(result.get(result.size() - 1), is(expectedFinalResult)); + } + /* * Runs the actual test. Checks the result after each input record to ensure fixed processing order. * If an input tuple does not trigger any result, "expectedResult" should contain a "null" entry @@ -190,7 +181,9 @@ void runTest(final List> expectedResult) throws Exception { assert expectedResult.size() == input.size(); IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); - final KafkaStreams streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG), TIME); + final KafkaStreams streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); + + System.out.println(builder.build().describe()); try { streams.start(); @@ -200,10 +193,8 @@ void runTest(final List> expectedResult) throws Exception { final Iterator> resultIterator = expectedResult.iterator(); for (final Input singleInput : input) { producer.send(new ProducerRecord<>(singleInput.topic, null, ++ts, singleInput.record.key, singleInput.record.value)).get(); - TIME.sleep(COMMIT_INTERVAL + 1); List expected = resultIterator.next(); - System.out.println("checking result: " + expected); checkResult(OUTPUT_TOPIC, expected); } @@ -212,6 +203,37 @@ void runTest(final List> expectedResult) throws Exception { } } + /* + * Runs the actual test. Checks the final result only after expected number of records have been consumed. + */ + void runTest(final String expectedFinalResult) throws Exception { + IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); + final KafkaStreams streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); + + try { + streams.start(); + + long ts = System.currentTimeMillis(); + + for (final Input singleInput : input) { + producer.send(new ProducerRecord<>(singleInput.topic, null, ++ts, singleInput.record.key, singleInput.record.value)).get(); + } + + TestUtils.waitForCondition(new TestCondition() { + @Override + public boolean conditionMet() { + System.out.println("RESULT: " + finalResultReached.get()); + return finalResultReached.get(); + } + }, "Never received expected final result."); + + checkResult(OUTPUT_TOPIC, expectedFinalResult, numRecordsExpected); + + } finally { + streams.close(); + } + } + private final class Input { String topic; KeyValue record; diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java index 9f8ace22c9846..ba581f0fcefaf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -16,49 +16,20 @@ */ package org.apache.kafka.streams.integration; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; -import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; -import org.apache.kafka.streams.kstream.JoinWindows; -import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.MockKeyValueMapper; -import org.apache.kafka.test.TestUtils; -import org.junit.After; import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.Iterator; import java.util.List; -import java.util.Properties; - -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.Is.is; /** * Tests all available joins of Kafka Streams DSL. @@ -84,52 +55,39 @@ public void prepareTopology() throws InterruptedException { rightTable = builder.table(INPUT_TOPIC_RIGHT); } - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - private List> dedupExpectedJoinResult = Arrays.asList( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - Arrays.asList("D-d", "D-d") - ); - - private List> dedupExpectedMultiJoinResult = Arrays.asList( - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - null, - Arrays.asList("D-d-d", "D-d-d") - ); + final private String expectedFinalJoinResult = "D-d"; + final private String expectedFinalMultiJoinResult = "D-d-d"; + + final private class CountingPeek implements ForeachAction { + final private String expected; + + CountingPeek(final boolean multiJoin) { + this.expected = multiJoin ? expectedFinalMultiJoinResult : expectedFinalJoinResult; + } + + @Override + public void apply(final Long key, final String value) { + numRecordsExpected++; + if (value.equals(expected)) { + boolean ret = finalResultReached.compareAndSet(false, true); + + if (!ret) { + // do nothing; it is possible that we will see multiple duplicates of final results due to KAFKA-4309 + // TODO: should be removed when KAFKA-4309 is fixed + } + } + } + } @Test public void testInner() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner"); - List> expectedResult; if (cacheEnabled) { - expectedResult = dedupExpectedJoinResult; + leftTable.join(rightTable, valueJoiner).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); + runTest(expectedFinalJoinResult); } else { - expectedResult = Arrays.asList( + List> expectedResult = Arrays.asList( null, null, null, @@ -146,22 +104,21 @@ public void testInner() throws Exception { null, Collections.singletonList("D-d") ); - } - - leftTable.join(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + leftTable.join(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); + runTest(expectedResult); + } } @Test public void testLeft() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left"); - List> expectedResult; if (cacheEnabled) { - expectedResult = dedupExpectedJoinResult; + leftTable.leftJoin(rightTable, valueJoiner).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); + runTest(expectedFinalJoinResult); } else { - expectedResult = Arrays.asList( + List> expectedResult = Arrays.asList( null, null, Collections.singletonList("A-null"), @@ -178,23 +135,21 @@ public void testLeft() throws Exception { null, Collections.singletonList("D-d") ); - } - leftTable.leftJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - - runTest(expectedResult); + leftTable.leftJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); + runTest(expectedResult); + } } @Test public void testOuter() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - List> expectedResult; if (cacheEnabled) { - expectedResult = dedupExpectedJoinResult; + leftTable.outerJoin(rightTable, valueJoiner).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); + runTest(expectedFinalJoinResult); } else { - expectedResult = Arrays.asList( + List> expectedResult = Arrays.asList( null, null, Collections.singletonList("A-null"), @@ -211,121 +166,130 @@ public void testOuter() throws Exception { Collections.singletonList("null-d"), Collections.singletonList("D-d") ); - } - leftTable.outerJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - - runTest(expectedResult); + leftTable.outerJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); + runTest(expectedResult); + } } @Test public void testInnerInner() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-inner"); - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - List> expectedResult; if (cacheEnabled) { - expectedResult = dedupExpectedMultiJoinResult; + leftTable.join(rightTable, valueJoiner) + .join(rightTable, valueJoiner) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult); } else { - expectedResult = Arrays.asList( + List> expectedResult = Arrays.asList( null, null, null, - Collections.singletonList("A-a-a"), + Arrays.asList("A-a-a", "A-a-a"), Collections.singletonList("B-a-a"), - Collections.singletonList("B-b-b"), + Arrays.asList("B-b-b", "B-b-b"), Collections.singletonList((String) null), null, null, - Collections.singletonList("C-c-c"), - Collections.singletonList((String) null), + Arrays.asList("C-c-c", "C-c-c"), + null, null, null, null, Collections.singletonList("D-d-d") ); - } - leftTable.join(rightTable, valueJoiner) - .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) - .reduce(reducer, reducer) - .join(rightTable, valueJoiner) - .toStream().to(OUTPUT_TOPIC); + leftTable.join(rightTable, valueJoiner) + .join(rightTable, valueJoiner) + .toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + runTest(expectedResult); + } } @Test public void testInnerLeft() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-left"); - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - List> expectedResult; if (cacheEnabled) { - expectedResult = dedupExpectedMultiJoinResult; + leftTable.join(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult); } else { - expectedResult = Arrays.asList( + List> expectedResult = Arrays.asList( null, null, null, - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), + Arrays.asList("A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), Collections.singletonList((String) null), null, null, - Collections.singletonList("C-c"), - Collections.singletonList((String) null), + Arrays.asList("C-c-c", "C-c-c"), + Arrays.asList((String) null, null), null, null, null, - Collections.singletonList("D-d") + Collections.singletonList("D-d-d") ); - } - - leftTable.join(rightTable, valueJoiner) - .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) - .reduce(reducer, reducer) - .leftJoin(rightTable, valueJoiner) - .toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + leftTable.join(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner) + .toStream() + .peek(new ForeachAction() { + @Override + public void apply(Long key, String value) { + System.out.println("OUTPUT: " + key + "->" + value); + } + }) + .to(OUTPUT_TOPIC); + + runTest(expectedResult); + } } @Test public void testInnerOuter() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-outer"); - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - List> expectedResult; if (cacheEnabled) { - expectedResult = dedupExpectedMultiJoinResult; + leftTable.join(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult); } else { - expectedResult = Arrays.asList( + List> expectedResult = Arrays.asList( null, null, null, - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), + Arrays.asList("A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), + Collections.singletonList("null-b"), Collections.singletonList((String) null), null, - null, - Collections.singletonList("C-c"), - Collections.singletonList((String) null), + Arrays.asList("C-c-c", "C-c-c"), + Arrays.asList((String) null, null), null, null, null, - Collections.singletonList("D-d") + Arrays.asList("null-d", "D-d-d") ); - } - leftTable.join(rightTable, valueJoiner) - .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) - .reduce(reducer, reducer) - .leftJoin(rightTable, valueJoiner) - .toStream().to(OUTPUT_TOPIC); + leftTable.join(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner) + .toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + runTest(expectedResult); + } } } From d49538659971430bbe74a5a31dc8026002586a25 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 09:52:20 -0800 Subject: [PATCH 06/16] Add more tests to table table join --- .../AbstractJoinIntegrationTest.java | 69 +++- .../integration/JoinIntegrationTest.java | 46 +-- .../TableTableJoinIntegrationTest.java | 312 ++++++++++++++++-- .../org/apache/kafka/test/MockMapper.java | 3 +- 4 files changed, 349 insertions(+), 81 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index 0cd81ecb98145..2a08562cc218b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -32,6 +32,9 @@ import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.QueryableStoreTypes; +import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.TestCondition; import org.apache.kafka.test.TestUtils; @@ -82,11 +85,13 @@ public static Collection data() { static final String INPUT_TOPIC_RIGHT = "inputTopicRight"; static final String INPUT_TOPIC_LEFT = "inputTopicLeft"; static final String OUTPUT_TOPIC = "outputTopic"; + private final long anyUniqueKey = 0L; private final static Properties PRODUCER_CONFIG = new Properties(); private final static Properties RESULT_CONSUMER_CONFIG = new Properties(); private KafkaProducer producer; + private KafkaStreams streams; StreamsBuilder builder; int numRecordsExpected = 0; @@ -162,10 +167,8 @@ public void cleanup() throws InterruptedException { } private void checkResult(final String outputTopic, final List expectedResult) throws InterruptedException { - if (expectedResult != null) { - final List result = IntegrationTestUtils.waitUntilMinValuesRecordsReceived(RESULT_CONSUMER_CONFIG, outputTopic, expectedResult.size(), 30 * 1000L); - assertThat(result, is(expectedResult)); - } + final List result = IntegrationTestUtils.waitUntilMinValuesRecordsReceived(RESULT_CONSUMER_CONFIG, outputTopic, expectedResult.size(), 30 * 1000L); + assertThat(result, is(expectedResult)); } private void checkResult(final String outputTopic, final String expectedFinalResult, final int expectedTotalNumRecords) throws InterruptedException { @@ -178,12 +181,23 @@ private void checkResult(final String outputTopic, final String expectedFinalRes * If an input tuple does not trigger any result, "expectedResult" should contain a "null" entry */ void runTest(final List> expectedResult) throws Exception { + runTest(expectedResult, null); + } + + + /* + * Runs the actual test. Checks the result after each input record to ensure fixed processing order. + * If an input tuple does not trigger any result, "expectedResult" should contain a "null" entry + */ + void runTest(final List> expectedResult, final String storeName) throws Exception { assert expectedResult.size() == input.size(); + System.out.println(builder.build().describe()); + IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); - final KafkaStreams streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); + streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); - System.out.println(builder.build().describe()); + String expectedFinalResult = null; try { streams.start(); @@ -196,19 +210,33 @@ void runTest(final List> expectedResult) throws Exception { List expected = resultIterator.next(); - checkResult(OUTPUT_TOPIC, expected); + if (expected != null) { + checkResult(OUTPUT_TOPIC, expected); + expectedFinalResult = expected.get(expected.size() - 1); + } } + + if (storeName != null) + checkQueryableStore(storeName, expectedFinalResult); + } finally { streams.close(); } } /* - * Runs the actual test. Checks the final result only after expected number of records have been consumed. - */ + * Runs the actual test. Checks the final result only after expected number of records have been consumed. + */ void runTest(final String expectedFinalResult) throws Exception { + runTest(expectedFinalResult, null); + } + + /* + * Runs the actual test. Checks the final result only after expected number of records have been consumed. + */ + void runTest(final String expectedFinalResult, final String storeName) throws Exception { IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); - final KafkaStreams streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); + streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); try { streams.start(); @@ -229,17 +257,34 @@ public boolean conditionMet() { checkResult(OUTPUT_TOPIC, expectedFinalResult, numRecordsExpected); + if (storeName != null) + checkQueryableStore(storeName, expectedFinalResult); + } finally { streams.close(); } } + /* + * Checks the embedded queryable state store snapshot + */ + private void checkQueryableStore(final String queryableName, final String expectedFinalResult) { + final ReadOnlyKeyValueStore store = streams.store(queryableName, QueryableStoreTypes.keyValueStore()); + + final KeyValueIterator all = store.all(); + final KeyValue onlyEntry = all.next(); + + assertThat(onlyEntry.key, is(anyUniqueKey)); + assertThat(onlyEntry.value, is(expectedFinalResult)); + assertThat(all.hasNext(), is(false)); + + all.close(); + } + private final class Input { String topic; KeyValue record; - private final long anyUniqueKey = 0L; - Input(final String topic, final V value) { this.topic = topic; record = KeyValue.pair(anyUniqueKey, value); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java index a2cf39c1f4082..d72a439332545 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java @@ -16,49 +16,23 @@ */ package org.apache.kafka.streams.integration; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.KafkaProducer; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; -import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.MockKeyValueMapper; -import org.apache.kafka.test.TestUtils; -import org.junit.After; +import org.apache.kafka.test.MockMapper; import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Rule; import org.junit.Test; import org.junit.experimental.categories.Category; -import org.junit.rules.TemporaryFolder; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collection; import java.util.Collections; -import java.util.Iterator; import java.util.List; -import java.util.Properties; -import static org.hamcrest.MatcherAssert.assertThat; -import static org.hamcrest.core.Is.is; /** * Tests all available joins of Kafka Streams DSL. @@ -137,9 +111,9 @@ public void testInnerRepartitionedKStreamKStream() throws Exception { Arrays.asList("D-a", "D-b", "D-c", "D-d") ); - leftStream.map(MockKeyValueMapper.NoOpKeyValueMapper()) - .join(rightStream.flatMap(MockKeyValueMapper.NoOpFlatKeyValueMapper()) - .selectKey(MockKeyValueMapper.SelectKeyKeyValueMapper()), + leftStream.map(MockMapper.noOpKeyValueMapper()) + .join(rightStream.flatMap(MockMapper.noOpFlatKeyValueMapper()) + .selectKey(MockMapper.selectKeyKeyValueMapper()), valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); runTest(expectedResult); @@ -194,9 +168,9 @@ public void testLeftRepartitionedKStreamKStream() throws Exception { Arrays.asList("D-a", "D-b", "D-c", "D-d") ); - leftStream.map(MockKeyValueMapper.NoOpKeyValueMapper()) - .leftJoin(rightStream.flatMap(MockKeyValueMapper.NoOpFlatKeyValueMapper()) - .selectKey(MockKeyValueMapper.SelectKeyKeyValueMapper()), + leftStream.map(MockMapper.noOpKeyValueMapper()) + .leftJoin(rightStream.flatMap(MockMapper.noOpFlatKeyValueMapper()) + .selectKey(MockMapper.selectKeyKeyValueMapper()), valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); runTest(expectedResult); @@ -251,9 +225,9 @@ public void testOuterRepartitionedKStreamKStream() throws Exception { Arrays.asList("D-a", "D-b", "D-c", "D-d") ); - leftStream.map(MockKeyValueMapper.NoOpKeyValueMapper()) - .outerJoin(rightStream.flatMap(MockKeyValueMapper.NoOpFlatKeyValueMapper()) - .selectKey(MockKeyValueMapper.SelectKeyKeyValueMapper()), + leftStream.map(MockMapper.noOpKeyValueMapper()) + .outerJoin(rightStream.flatMap(MockMapper.noOpFlatKeyValueMapper()) + .selectKey(MockMapper.selectKeyKeyValueMapper()), valueJoiner, JoinWindows.of(10000)).to(OUTPUT_TOPIC); runTest(expectedResult); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java index ba581f0fcefaf..c0b7a555763c0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -16,10 +16,14 @@ */ package org.apache.kafka.streams.integration; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.StreamsBuilder; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Materialized; +import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.test.IntegrationTest; import org.junit.Before; import org.junit.Test; @@ -57,6 +61,13 @@ public void prepareTopology() throws InterruptedException { final private String expectedFinalJoinResult = "D-d"; final private String expectedFinalMultiJoinResult = "D-d-d"; + final private String storeName = APP_ID + "-store"; + + private Materialized> materialized = Materialized.>as(storeName) + .withKeySerde(Serdes.Long()) + .withValueSerde(Serdes.String()) + .withCachingDisabled() + .withLoggingDisabled(); final private class CountingPeek implements ForeachAction { final private String expected; @@ -84,8 +95,8 @@ public void testInner() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner"); if (cacheEnabled) { - leftTable.join(rightTable, valueJoiner).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); - runTest(expectedFinalJoinResult); + leftTable.join(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); + runTest(expectedFinalJoinResult, storeName); } else { List> expectedResult = Arrays.asList( null, @@ -105,8 +116,8 @@ public void testInner() throws Exception { Collections.singletonList("D-d") ); - leftTable.join(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + leftTable.join(rightTable, valueJoiner, materialized).toStream().to(OUTPUT_TOPIC); + runTest(expectedResult, storeName); } } @@ -115,8 +126,8 @@ public void testLeft() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left"); if (cacheEnabled) { - leftTable.leftJoin(rightTable, valueJoiner).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); - runTest(expectedFinalJoinResult); + leftTable.leftJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); + runTest(expectedFinalJoinResult, storeName); } else { List> expectedResult = Arrays.asList( null, @@ -136,8 +147,8 @@ public void testLeft() throws Exception { Collections.singletonList("D-d") ); - leftTable.leftJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + leftTable.leftJoin(rightTable, valueJoiner, materialized).toStream().to(OUTPUT_TOPIC); + runTest(expectedResult, storeName); } } @@ -146,8 +157,8 @@ public void testOuter() throws Exception { STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); if (cacheEnabled) { - leftTable.outerJoin(rightTable, valueJoiner).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); - runTest(expectedFinalJoinResult); + leftTable.outerJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); + runTest(expectedFinalJoinResult, storeName); } else { List> expectedResult = Arrays.asList( null, @@ -167,8 +178,8 @@ public void testOuter() throws Exception { Collections.singletonList("D-d") ); - leftTable.outerJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + leftTable.outerJoin(rightTable, valueJoiner, materialized).toStream().to(OUTPUT_TOPIC); + runTest(expectedResult, storeName); } } @@ -178,11 +189,11 @@ public void testInnerInner() throws Exception { if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) - .join(rightTable, valueJoiner) + .join(rightTable, valueJoiner, materialized) .toStream() .peek(new CountingPeek(true)) .to(OUTPUT_TOPIC); - runTest(expectedFinalMultiJoinResult); + runTest(expectedFinalMultiJoinResult, storeName); } else { List> expectedResult = Arrays.asList( null, @@ -203,10 +214,10 @@ public void testInnerInner() throws Exception { ); leftTable.join(rightTable, valueJoiner) - .join(rightTable, valueJoiner) + .join(rightTable, valueJoiner, materialized) .toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + runTest(expectedResult, storeName); } } @@ -216,11 +227,11 @@ public void testInnerLeft() throws Exception { if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) - .leftJoin(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner, materialized) .toStream() .peek(new CountingPeek(true)) .to(OUTPUT_TOPIC); - runTest(expectedFinalMultiJoinResult); + runTest(expectedFinalMultiJoinResult, storeName); } else { List> expectedResult = Arrays.asList( null, @@ -233,7 +244,7 @@ public void testInnerLeft() throws Exception { null, null, Arrays.asList("C-c-c", "C-c-c"), - Arrays.asList((String) null, null), + Collections.singletonList((String) null), null, null, null, @@ -241,17 +252,11 @@ public void testInnerLeft() throws Exception { ); leftTable.join(rightTable, valueJoiner) - .leftJoin(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner, materialized) .toStream() - .peek(new ForeachAction() { - @Override - public void apply(Long key, String value) { - System.out.println("OUTPUT: " + key + "->" + value); - } - }) .to(OUTPUT_TOPIC); - runTest(expectedResult); + runTest(expectedResult, storeName); } } @@ -261,11 +266,11 @@ public void testInnerOuter() throws Exception { if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) - .outerJoin(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner, materialized) .toStream() .peek(new CountingPeek(true)) .to(OUTPUT_TOPIC); - runTest(expectedFinalMultiJoinResult); + runTest(expectedFinalMultiJoinResult, storeName); } else { List> expectedResult = Arrays.asList( null, @@ -286,10 +291,255 @@ public void testInnerOuter() throws Exception { ); leftTable.join(rightTable, valueJoiner) - .outerJoin(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner, materialized) + .toStream().to(OUTPUT_TOPIC); + + runTest(expectedResult, storeName); + } + } + + @Test + public void testLeftInner() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-inner"); + + if (cacheEnabled) { + leftTable.leftJoin(rightTable, valueJoiner) + .join(rightTable, valueJoiner, materialized) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult, storeName); + } else { + List> expectedResult = Arrays.asList( + null, + null, + null, + Arrays.asList("A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), + Collections.singletonList((String) null), + null, + null, + Arrays.asList("C-c-c", "C-c-c"), + Collections.singletonList((String) null), + null, + null, + null, + Collections.singletonList("D-d-d") + ); + + leftTable.leftJoin(rightTable, valueJoiner) + .join(rightTable, valueJoiner, materialized) + .toStream() + .to(OUTPUT_TOPIC); + + runTest(expectedResult, storeName); + } + } + + @Test + public void testLeftLeft() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-left"); + + if (cacheEnabled) { + leftTable.leftJoin(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner, materialized) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult, storeName); + } else { + List> expectedResult = Arrays.asList( + null, + null, + null, + Arrays.asList("A-null-null", "A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), + Collections.singletonList((String) null), + null, + null, + Arrays.asList("C-null-null", "C-c-c", "C-c-c"), + Arrays.asList("C-null-null", "C-null-null"), + Collections.singletonList((String) null), + null, + null, + Collections.singletonList("D-d-d") + ); + + leftTable.leftJoin(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner, materialized) + .toStream() + .to(OUTPUT_TOPIC); + + runTest(expectedResult, storeName); + } + } + + @Test + public void testLeftOuter() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-outer"); + + if (cacheEnabled) { + leftTable.leftJoin(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner, materialized) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult, storeName); + } else { + List> expectedResult = Arrays.asList( + null, + null, + null, + Arrays.asList("A-null-null", "A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), + Collections.singletonList("null-b-b"), + Collections.singletonList((String) null), + null, + Arrays.asList("C-null-null", "C-c-c", "C-c-c"), + Arrays.asList("C-null-null", "C-null-null"), + Collections.singletonList((String) null), + null, + null, + Arrays.asList("null-d", "D-d-d") + ); + + leftTable.leftJoin(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner, materialized) .toStream().to(OUTPUT_TOPIC); - runTest(expectedResult); + runTest(expectedResult, storeName); + } + } + + @Test + public void testOuterInner() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-inner"); + + if (cacheEnabled) { + leftTable.outerJoin(rightTable, valueJoiner) + .join(rightTable, valueJoiner, materialized) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult, storeName); + } else { + List> expectedResult = Arrays.asList( + null, + null, + null, + Arrays.asList("A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), + Collections.singletonList("null-b-b"), + null, + null, + Arrays.asList("C-c-c", "C-c-c"), + Collections.singletonList((String) null), + null, + null, + Arrays.asList("null-d-d", "null-d-d"), + Collections.singletonList("D-d-d") + ); + + leftTable.outerJoin(rightTable, valueJoiner) + .join(rightTable, valueJoiner, materialized) + .toStream() + .to(OUTPUT_TOPIC); + + runTest(expectedResult, storeName); + } + } + + @Test + public void testOuterLeft() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-left"); + + if (cacheEnabled) { + leftTable.outerJoin(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner, materialized) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult, storeName); + } else { + List> expectedResult = Arrays.asList( + null, + null, + null, + Arrays.asList("A-null-null", "A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), + Collections.singletonList("null-b-b"), + Collections.singletonList((String) null), + null, + Arrays.asList("C-null-null", "C-c-c", "C-c-c"), + Arrays.asList("C-null-null", "C-null-null"), + Collections.singletonList((String) null), + null, + Arrays.asList("null-d-d", "null-d-d"), + Collections.singletonList("D-d-d") + ); + + leftTable.outerJoin(rightTable, valueJoiner) + .leftJoin(rightTable, valueJoiner, materialized) + .toStream() + .peek(new ForeachAction() { + @Override + public void apply(Long key, String value) { + System.out.println("OUTPUT: " + key + "->" + value); + } + }) + .to(OUTPUT_TOPIC); + + runTest(expectedResult, storeName); + } + } + + @Test + public void testOuterOuter() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-outer"); + + if (cacheEnabled) { + leftTable.outerJoin(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner, materialized) + .toStream() + .peek(new CountingPeek(true)) + .to(OUTPUT_TOPIC); + runTest(expectedFinalMultiJoinResult, storeName); + } else { + List> expectedResult = Arrays.asList( + null, + null, + null, + Arrays.asList("A-null-null", "A-a-a", "A-a-a"), + Collections.singletonList("B-a-a"), + Arrays.asList("B-b-b", "B-b-b"), + Collections.singletonList("null-b"), + Collections.singletonList((String) null), + null, + Arrays.asList("C-null-null", "C-c-c", "C-c-c"), + Arrays.asList("C-null-null", "C-null-null"), + Collections.singletonList((String) null), + null, + null, + Arrays.asList("null-d", "D-d-d") + ); + + leftTable.outerJoin(rightTable, valueJoiner) + .outerJoin(rightTable, valueJoiner, materialized) + .toStream() + .peek(new ForeachAction() { + @Override + public void apply(Long key, String value) { + System.out.println("OUTPUT: " + key + "->" + value); + } + }) + .to(OUTPUT_TOPIC); + + runTest(expectedResult, storeName); } } } diff --git a/streams/src/test/java/org/apache/kafka/test/MockMapper.java b/streams/src/test/java/org/apache/kafka/test/MockMapper.java index a03c76b24c5e6..518419984f183 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockMapper.java +++ b/streams/src/test/java/org/apache/kafka/test/MockMapper.java @@ -70,8 +70,7 @@ public static KeyValueMapper selectKeyKeyValueMapper() { return new SelectKeyMapper<>(); } - - public static KeyValueMapper>> NoOpFlatKeyValueMapper() { + public static KeyValueMapper>> noOpFlatKeyValueMapper() { return new NoOpFlatKeyValueMapper<>(); } From a7bb2e87c2e621b705fc64dc1af408026e9cd87a Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 10:17:35 -0800 Subject: [PATCH 07/16] minor fixes --- .../TableTableJoinIntegrationTest.java | 20 ++++--------------- 1 file changed, 4 insertions(+), 16 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java index c0b7a555763c0..2dd86e2970639 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -395,7 +395,7 @@ public void testLeftOuter() throws Exception { Arrays.asList("A-null-null", "A-a-a", "A-a-a"), Collections.singletonList("B-a-a"), Arrays.asList("B-b-b", "B-b-b"), - Collections.singletonList("null-b-b"), + Collections.singletonList("null-b"), Collections.singletonList((String) null), null, Arrays.asList("C-null-null", "C-c-c", "C-c-c"), @@ -486,12 +486,6 @@ public void testOuterLeft() throws Exception { leftTable.outerJoin(rightTable, valueJoiner) .leftJoin(rightTable, valueJoiner, materialized) .toStream() - .peek(new ForeachAction() { - @Override - public void apply(Long key, String value) { - System.out.println("OUTPUT: " + key + "->" + value); - } - }) .to(OUTPUT_TOPIC); runTest(expectedResult, storeName); @@ -517,26 +511,20 @@ public void testOuterOuter() throws Exception { Arrays.asList("A-null-null", "A-a-a", "A-a-a"), Collections.singletonList("B-a-a"), Arrays.asList("B-b-b", "B-b-b"), - Collections.singletonList("null-b"), - Collections.singletonList((String) null), + Collections.singletonList("null-b-b"), + Arrays.asList((String) null, null), null, Arrays.asList("C-null-null", "C-c-c", "C-c-c"), Arrays.asList("C-null-null", "C-null-null"), Collections.singletonList((String) null), null, null, - Arrays.asList("null-d", "D-d-d") + Arrays.asList("null-d-d", "null-d-d", "D-d-d") ); leftTable.outerJoin(rightTable, valueJoiner) .outerJoin(rightTable, valueJoiner, materialized) .toStream() - .peek(new ForeachAction() { - @Override - public void apply(Long key, String value) { - System.out.println("OUTPUT: " + key + "->" + value); - } - }) .to(OUTPUT_TOPIC); runTest(expectedResult, storeName); From 33ef9386d8859bbf27b6f7e6e62ad27895a7d79a Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 10:24:24 -0800 Subject: [PATCH 08/16] remove debugging prints --- .../kafka/streams/kstream/internals/KTableKTableJoin.java | 5 ----- .../streams/kstream/internals/KTableKTableLeftJoin.java | 2 -- .../apache/kafka/streams/kstream/internals/KTableSource.java | 3 --- .../kstream/internals/KTableSourceValueGetterSupplier.java | 2 -- 4 files changed, 12 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java index 892aaed010f0a..1b26a5b00dc2e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java @@ -84,10 +84,7 @@ public void process(final K key, final Change change) { R oldValue = null; final V2 value2 = valueGetter.get(key); - if (value2 == null) { - System.out.println("KTABLE JOIN " + change + " WITH " + value2 + ": NO RESULT"); - return; } @@ -99,8 +96,6 @@ public void process(final K key, final Change change) { oldValue = joiner.apply(change.oldValue, value2); } - System.out.println("KTABLE JOIN " + change + " WITH " + value2 + ": RESULT " + new Change<>(newValue, oldValue)); - context().forward(key, new Change<>(newValue, oldValue)); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 2e2f519486de6..fa57914ee8117 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -75,8 +75,6 @@ public void process(final K key, final Change change) { final V2 value2 = valueGetter.get(key); - System.out.println("KTABLE LEFT-JOIN: " + change + " v.s. " + value2); - if (value2 == null && change.newValue == null && change.oldValue == null) { return; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java index a404915cf98ae..de8bf2e7bb60f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSource.java @@ -60,9 +60,6 @@ public void process(K key, V value) { if (key == null) return; V oldValue = store.get(key); - - System.out.println("PUT " + key + "<-" + value + " in store " + storeName); - store.put(key, value); tupleForwarder.maybeForward(key, value, oldValue); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java index 5322e62ad42b2..714b8c50dfa37 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableSourceValueGetterSupplier.java @@ -46,8 +46,6 @@ public void init(ProcessorContext context) { } public V get(K key) { - System.out.println("GET " + key + "<-" + store.get(key) + " from store " + storeName); - return store.get(key); } From 2c33f1750f3dd929285e9d2b80a6a7f33ee56c88 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 10:25:20 -0800 Subject: [PATCH 09/16] remove old table table join integration test --- .../KTableKTableJoinIntegrationTest.java | 399 ------------------ 1 file changed, 399 deletions(-) delete mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java deleted file mode 100644 index a12ffac58ab57..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/integration/KTableKTableJoinIntegrationTest.java +++ /dev/null @@ -1,399 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.integration; - -import kafka.utils.MockTime; -import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.utils.Bytes; -import org.apache.kafka.streams.KafkaStreams; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.StreamsBuilder; -import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.integration.utils.EmbeddedKafkaCluster; -import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; -import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.kstream.Materialized; -import org.apache.kafka.streams.kstream.ValueJoiner; -import org.apache.kafka.streams.state.KeyValueIterator; -import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.QueryableStoreTypes; -import org.apache.kafka.streams.state.ReadOnlyKeyValueStore; -import org.apache.kafka.test.IntegrationTest; -import org.apache.kafka.test.TestUtils; -import org.junit.After; -import org.junit.Before; -import org.junit.BeforeClass; -import org.junit.ClassRule; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.io.IOException; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; - -import static org.hamcrest.CoreMatchers.equalTo; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertThat; -import static org.junit.Assert.assertTrue; - -@Category({IntegrationTest.class}) -public class KTableKTableJoinIntegrationTest { - private final static int NUM_BROKERS = 1; - - @ClassRule - public final static EmbeddedKafkaCluster CLUSTER = new EmbeddedKafkaCluster(NUM_BROKERS); - private final static MockTime MOCK_TIME = CLUSTER.time; - private final static String TABLE_1 = "table1"; - private final static String TABLE_2 = "table2"; - private final static String TABLE_3 = "table3"; - private final static String OUTPUT = "output-"; - private static Properties streamsConfig; - private KafkaStreams streams; - private final static Properties CONSUMER_CONFIG = new Properties(); - - @BeforeClass - public static void beforeTest() throws Exception { - CLUSTER.createTopics(TABLE_1, TABLE_2, TABLE_3, OUTPUT); - - streamsConfig = new Properties(); - streamsConfig.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - streamsConfig.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); - streamsConfig.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); - streamsConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - streamsConfig.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getPath()); - streamsConfig.put(StreamsConfig.CACHE_MAX_BYTES_BUFFERING_CONFIG, 0); - streamsConfig.put(IntegrationTestUtils.INTERNAL_LEAVE_GROUP_ON_CLOSE, true); - streamsConfig.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 100); - - final Properties producerConfig = new Properties(); - producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - producerConfig.put(ProducerConfig.ACKS_CONFIG, "all"); - producerConfig.put(ProducerConfig.RETRIES_CONFIG, 0); - producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - - final List> table1 = Arrays.asList( - new KeyValue<>("a", "A1"), - new KeyValue<>("b", "B1") - ); - - final List> table2 = Arrays.asList( - new KeyValue<>("b", "B2"), - new KeyValue<>("c", "C2") - ); - - final List> table3 = Arrays.asList( - new KeyValue<>("a", "A3"), - new KeyValue<>("b", "B3"), - new KeyValue<>("c", "C3") - ); - - // put table 3 first, to make sure data is there when joining T1 with T2 - IntegrationTestUtils.produceKeyValuesSynchronously(TABLE_3, table3, producerConfig, MOCK_TIME); - IntegrationTestUtils.produceKeyValuesSynchronously(TABLE_1, table1, producerConfig, MOCK_TIME); - IntegrationTestUtils.produceKeyValuesSynchronously(TABLE_2, table2, producerConfig, MOCK_TIME); - - CONSUMER_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, "ktable-ktable-consumer"); - CONSUMER_CONFIG.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - } - - @Before - public void before() throws IOException { - IntegrationTestUtils.purgeLocalStreamsState(streamsConfig); - } - - @After - public void after() throws IOException { - if (streams != null) { - streams.close(); - streams = null; - } - IntegrationTestUtils.purgeLocalStreamsState(streamsConfig); - } - - private enum JoinType { - INNER, LEFT, OUTER - } - - - @Test - public void shouldInnerInnerJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.INNER, JoinType.INNER, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), false); - } - - @Test - public void shouldInnerInnerJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.INNER, JoinType.INNER, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), true); - } - - @Test - public void shouldInnerLeftJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.INNER, JoinType.LEFT, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), false); - } - - @Test - public void shouldInnerLeftJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.INNER, JoinType.LEFT, Collections.singletonList(new KeyValue<>("b", "B1-B2-B3")), true); - } - - @Test - public void shouldInnerOuterJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.INNER, JoinType.OUTER, Arrays.asList( - new KeyValue<>("a", "null-A3"), - new KeyValue<>("b", "null-B3"), - new KeyValue<>("c", "null-C3"), - new KeyValue<>("b", "B1-B2-B3")), false); - } - - @Test - public void shouldInnerOuterJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.INNER, JoinType.OUTER, Arrays.asList( - new KeyValue<>("a", "null-A3"), - new KeyValue<>("b", "null-B3"), - new KeyValue<>("c", "null-C3"), - new KeyValue<>("b", "B1-B2-B3")), true); - } - - @Test - public void shouldLeftInnerJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.LEFT, JoinType.INNER, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3")), false); - } - - @Test - public void shouldLeftInnerJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.LEFT, JoinType.INNER, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3")), true); - } - - @Test - public void shouldLeftLeftJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.LEFT, JoinType.LEFT, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3")), false); - } - - @Test - public void shouldLeftLeftJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.LEFT, JoinType.LEFT, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3")), true); - } - - @Test - public void shouldLeftOuterJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.LEFT, JoinType.OUTER, Arrays.asList( - new KeyValue<>("a", "null-A3"), - new KeyValue<>("b", "null-B3"), - new KeyValue<>("c", "null-C3"), - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3")), false); - } - - @Test - public void shouldLeftOuterJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.LEFT, JoinType.OUTER, Arrays.asList( - new KeyValue<>("a", "null-A3"), - new KeyValue<>("b", "null-B3"), - new KeyValue<>("c", "null-C3"), - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3")), true); - } - - @Test - public void shouldOuterInnerJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.OUTER, JoinType.INNER, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3"), - new KeyValue<>("c", "null-C2-C3")), false); - } - - @Test - public void shouldOuterInnerJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.OUTER, JoinType.INNER, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3"), - new KeyValue<>("c", "null-C2-C3")), true); - } - - @Test - public void shouldOuterLeftJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.OUTER, JoinType.LEFT, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3"), - new KeyValue<>("c", "null-C2-C3")), false); - } - - @Test - public void shouldOuterLeftJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.OUTER, JoinType.LEFT, Arrays.asList( - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3"), - new KeyValue<>("c", "null-C2-C3")), true); - } - - @Test - public void shouldOuterOuterJoin() throws InterruptedException { - verifyKTableKTableJoin(JoinType.OUTER, JoinType.OUTER, Arrays.asList( - new KeyValue<>("a", "null-A3"), - new KeyValue<>("b", "null-B3"), - new KeyValue<>("c", "null-C3"), - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3"), - new KeyValue<>("c", "null-C2-C3")), false); - } - - @Test - public void shouldOuterOuterJoinQueryable() throws InterruptedException { - verifyKTableKTableJoin(JoinType.OUTER, JoinType.OUTER, Arrays.asList( - new KeyValue<>("a", "null-A3"), - new KeyValue<>("b", "null-B3"), - new KeyValue<>("c", "null-C3"), - new KeyValue<>("a", "A1-null-A3"), - new KeyValue<>("b", "B1-null-B3"), - new KeyValue<>("b", "B1-B2-B3"), - new KeyValue<>("c", "null-C2-C3")), true); - } - - - private void verifyKTableKTableJoin(final JoinType joinType1, - final JoinType joinType2, - final List> expectedResult, - boolean verifyQueryableState) throws InterruptedException { - final String queryableName = verifyQueryableState ? joinType1 + "-" + joinType2 + "-ktable-ktable-join-query" : null; - streamsConfig.put(StreamsConfig.APPLICATION_ID_CONFIG, joinType1 + "-" + joinType2 + "-ktable-ktable-join" + queryableName); - - streams = prepareTopology(joinType1, joinType2, queryableName); - streams.start(); - - final List> result = IntegrationTestUtils.waitUntilMinKeyValueRecordsReceived( - CONSUMER_CONFIG, - OUTPUT, - expectedResult.size()); - - assertThat(result, equalTo(expectedResult)); - - if (verifyQueryableState) { - verifyKTableKTableJoinQueryableState(joinType1, joinType2, expectedResult); - } - } - - private void verifyKTableKTableJoinQueryableState(final JoinType joinType1, - final JoinType joinType2, - final List> expectedResult) { - final String queryableName = joinType1 + "-" + joinType2 + "-ktable-ktable-join-query"; - final ReadOnlyKeyValueStore myJoinStore = streams.store(queryableName, - QueryableStoreTypes.keyValueStore()); - - // store only keeps last set of values, not entire stream of value changes - final Map expectedInStore = new HashMap<>(); - for (KeyValue expected : expectedResult) { - expectedInStore.put(expected.key, expected.value); - } - - for (Map.Entry expected : expectedInStore.entrySet()) { - assertEquals(expected.getValue(), myJoinStore.get(expected.getKey())); - } - final KeyValueIterator all = myJoinStore.all(); - while (all.hasNext()) { - KeyValue storeEntry = all.next(); - assertTrue(expectedResult.contains(storeEntry)); - } - all.close(); - - } - - private KafkaStreams prepareTopology(final JoinType joinType1, final JoinType joinType2, final String queryableName) { - final StreamsBuilder builder = new StreamsBuilder(); - - final KTable table1 = builder.table(TABLE_1); - final KTable table2 = builder.table(TABLE_2); - final KTable table3 = builder.table(TABLE_3); - - Materialized> materialized = null; - if (queryableName != null) { - materialized = Materialized.>as(queryableName) - .withKeySerde(Serdes.String()) - .withValueSerde(Serdes.String()) - .withCachingDisabled(); - } - join(join(table1, table2, joinType1, null /* no need to query intermediate result */), table3, - joinType2, materialized).to(OUTPUT); - - return new KafkaStreams(builder.build(), new StreamsConfig(streamsConfig)); - } - - private KTable join(final KTable first, - final KTable second, - final JoinType joinType, - final Materialized> materialized) { - final ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(final String value1, final String value2) { - return value1 + "-" + value2; - } - }; - - - switch (joinType) { - case INNER: - if (materialized != null) { - return first.join(second, joiner, materialized); - } else { - return first.join(second, joiner); - } - case LEFT: - if (materialized != null) { - return first.leftJoin(second, joiner, materialized); - } else { - return first.leftJoin(second, joiner); - } - case OUTER: - if (materialized != null) { - return first.outerJoin(second, joiner, materialized); - } else { - return first.outerJoin(second, joiner); - } - } - - throw new RuntimeException("Unknown join type."); - } - -} From e01a072a4ccc0f14a66ef6eeb8b830ad8e358e04 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 10:35:45 -0800 Subject: [PATCH 10/16] rename functions --- .../integration/JoinIntegrationTest.java | 238 ++---------------- 1 file changed, 16 insertions(+), 222 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java index d72a439332545..f7bd604f9ac78 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java @@ -20,7 +20,6 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; -import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.test.IntegrationTest; import org.apache.kafka.test.MockMapper; import org.junit.Before; @@ -42,8 +41,6 @@ public class JoinIntegrationTest extends AbstractJoinIntegrationTest { private KStream leftStream; private KStream rightStream; - private KTable leftTable; - private KTable rightTable; public JoinIntegrationTest(boolean cacheEnabled) { super(cacheEnabled); @@ -56,15 +53,13 @@ public void prepareTopology() throws InterruptedException { APP_ID = "stream-stream-join-integration-test"; builder = new StreamsBuilder(); - leftTable = builder.table(INPUT_TOPIC_LEFT); - rightTable = builder.table(INPUT_TOPIC_RIGHT); - leftStream = leftTable.toStream(); - rightStream = rightTable.toStream(); + leftStream = builder.stream(INPUT_TOPIC_LEFT); + rightStream = builder.stream(INPUT_TOPIC_RIGHT); } @Test - public void testInnerKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-KStream-KStream"); + public void testInner() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner"); final List> expectedResult = Arrays.asList( null, @@ -90,8 +85,8 @@ public void testInnerKStreamKStream() throws Exception { } @Test - public void testInnerRepartitionedKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-repartitioned-KStream-KStream"); + public void testInnerRepartitioned() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-repartitioned"); final List> expectedResult = Arrays.asList( null, @@ -120,8 +115,8 @@ public void testInnerRepartitionedKStreamKStream() throws Exception { } @Test - public void testLeftKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-KStream-KStream"); + public void testLeft() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left"); final List> expectedResult = Arrays.asList( null, @@ -147,8 +142,8 @@ public void testLeftKStreamKStream() throws Exception { } @Test - public void testLeftRepartitionedKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-repartitioned-KStream-KStream"); + public void testLeftRepartitioned() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-repartitioned"); final List> expectedResult = Arrays.asList( null, @@ -177,8 +172,8 @@ public void testLeftRepartitionedKStreamKStream() throws Exception { } @Test - public void testOuterKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer-KStream-KStream"); + public void testOuter() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); final List> expectedResult = Arrays.asList( null, @@ -204,8 +199,8 @@ public void testOuterKStreamKStream() throws Exception { } @Test - public void testOuterRepartitionedKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer-repartitioned-KStream-KStream"); + public void testOuterRepartitioned() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); final List> expectedResult = Arrays.asList( null, @@ -234,8 +229,8 @@ public void testOuterRepartitionedKStreamKStream() throws Exception { } @Test - public void testMultiInnerKStreamKStream() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-multi-inner-KStream-KStream"); + public void testMultiInner() throws Exception { + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-multi-inner"); final List> expectedResult = Arrays.asList( null, @@ -264,205 +259,4 @@ public void testMultiInnerKStreamKStream() throws Exception { runTest(expectedResult); } - - @Test - public void testInnerKStreamKTable() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-KStream-KTable"); - - final List> expectedResult = Arrays.asList( - null, - null, - null, - null, - Collections.singletonList("B-a"), - null, - null, - null, - null, - null, - null, - null, - null, - null, - Collections.singletonList("D-d") - ); - - leftStream.join(rightTable, valueJoiner).to(OUTPUT_TOPIC); - - runTest(expectedResult); - } - - @Test - public void testLeftKStreamKTable() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-KStream-KTable"); - - final List> expectedResult = Arrays.asList( - null, - null, - Collections.singletonList("A-null"), - null, - Collections.singletonList("B-a"), - null, - null, - null, - Collections.singletonList("C-null"), - null, - null, - null, - null, - null, - Collections.singletonList("D-d") - ); - - leftStream.leftJoin(rightTable, valueJoiner).to(OUTPUT_TOPIC); - - runTest(expectedResult); - } - - @Test - public void testInnerKTableKTable() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-KTable-KTable"); - - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - List> expectedResult; - if (cacheEnabled) { - expectedResult = Arrays.asList( - null, - null, - null, - Arrays.asList("A-a", "A-a"), // dup - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList((String) null), - null, - Collections.singletonList((String) null), // dup - Collections.singletonList("C-c"), - Collections.singletonList((String) null), - null, - null, - Collections.singletonList((String) null), // dup - Collections.singletonList("D-d") - ); - } else { - expectedResult = Arrays.asList( - null, - null, - null, - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList((String) null), - null, - null, - Collections.singletonList("C-c"), - Collections.singletonList((String) null), - null, - null, - null, - Collections.singletonList("D-d") - ); - } - - leftTable.join(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - - runTest(expectedResult); - } - - @Test - public void testLeftKTableKTable() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-KTable-KTable"); - - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - List> expectedResult; - if (cacheEnabled) { - expectedResult = Arrays.asList( - null, - null, - Arrays.asList("A-null", "A-null"), // dup - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList((String) null), - null, - Arrays.asList("C-null", "C-null"), // dup - Collections.singletonList("C-c"), - Collections.singletonList("C-null"), - Collections.singletonList((String) null), - null, - null, - Arrays.asList("D-d", "D-d") - ); - } else { - expectedResult = Arrays.asList( - null, - null, - Collections.singletonList("A-null"), - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList((String) null), - null, - Collections.singletonList("C-null"), - Collections.singletonList("C-c"), - Collections.singletonList("C-null"), - Collections.singletonList((String) null), - null, - null, - Collections.singletonList("D-d") - ); - } - - leftTable.leftJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - - runTest(expectedResult); - } - - @Test - public void testOuterKTableKTable() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer-KTable-KTable"); - - // TODO: the duplicate is due to KAFKA-4309, should be removed when it is fixed - List> expectedResult; - if (cacheEnabled) { - expectedResult = Arrays.asList( - null, - null, - Arrays.asList("A-null", "A-null"), - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList("null-b"), - Collections.singletonList((String) null), - Collections.singletonList("C-null"), - Collections.singletonList("C-c"), - Collections.singletonList("C-null"), - Collections.singletonList((String) null), - null, - Collections.singletonList("null-d"), - Collections.singletonList("D-d") - ); - } else { - expectedResult = Arrays.asList( - null, - null, - Collections.singletonList("A-null"), - Collections.singletonList("A-a"), - Collections.singletonList("B-a"), - Collections.singletonList("B-b"), - Collections.singletonList("null-b"), - Collections.singletonList((String) null), - Collections.singletonList("C-null"), - Collections.singletonList("C-c"), - Collections.singletonList("C-null"), - Collections.singletonList((String) null), - null, - Collections.singletonList("null-d"), - Collections.singletonList("D-d") - ); - } - - leftTable.outerJoin(rightTable, valueJoiner).toStream().to(OUTPUT_TOPIC); - - runTest(expectedResult); - } } From 1ee24db6e727a3e4eeb5c0655318c1e58435984d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 10:38:59 -0800 Subject: [PATCH 11/16] more renames and minor fixes --- .../kafka/streams/kstream/internals/KStreamKStreamJoin.java | 3 +-- .../kafka/streams/kstream/internals/KTableKTableLeftJoin.java | 1 - .../streams/kstream/internals/KTableKTableOuterJoin.java | 3 --- ...egrationTest.java => StreamStreamJoinIntegrationTest.java} | 4 ++-- 4 files changed, 3 insertions(+), 8 deletions(-) rename streams/src/test/java/org/apache/kafka/streams/integration/{JoinIntegrationTest.java => StreamStreamJoinIntegrationTest.java} (98%) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index 9e4f19429d1da..4f26767fe408c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -80,8 +80,7 @@ public void process(final K key, final V1 value) { try (WindowStoreIterator iter = otherWindow.fetch(key, timeFrom, timeTo)) { while (iter.hasNext()) { needOuterJoin = false; - V2 value2 = iter.next().value; - context().forward(key, joiner.apply(value, value2)); + context().forward(key, joiner.apply(value, iter.next().value)); } if (needOuterJoin) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index fa57914ee8117..c308a0dd36421 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -74,7 +74,6 @@ public void process(final K key, final Change change) { R oldValue = null; final V2 value2 = valueGetter.get(key); - if (value2 == null && change.newValue == null && change.oldValue == null) { return; } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 7649c152faa4d..9cee4f313182a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -73,9 +73,6 @@ public void process(final K key, final Change change) { R oldValue = null; final V2 value2 = valueGetter.get(key); - - System.out.println("KTABLE OUTER-JOIN: " + change + " v.s. " + value2); - if (value2 == null && change.newValue == null && change.oldValue == null) { return; } diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java similarity index 98% rename from streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java rename to streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java index f7bd604f9ac78..bd6abc31e735a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/JoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java @@ -38,11 +38,11 @@ */ @Category({IntegrationTest.class}) @RunWith(value = Parameterized.class) -public class JoinIntegrationTest extends AbstractJoinIntegrationTest { +public class StreamStreamJoinIntegrationTest extends AbstractJoinIntegrationTest { private KStream leftStream; private KStream rightStream; - public JoinIntegrationTest(boolean cacheEnabled) { + public StreamStreamJoinIntegrationTest(boolean cacheEnabled) { super(cacheEnabled); } From 62852172a9bae74644044bfbc1d9ace967d6454f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 10:53:56 -0800 Subject: [PATCH 12/16] fix style --- .../AbstractJoinIntegrationTest.java | 4 +-- .../StreamStreamJoinIntegrationTest.java | 16 +++++------ .../TableTableJoinIntegrationTest.java | 28 +++++++++---------- 3 files changed, 24 insertions(+), 24 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index 2a08562cc218b..594a7cd272c60 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -78,7 +78,7 @@ public static Collection data() { return values; } - static String APP_ID; + static String APPID; private static final Long COMMIT_INTERVAL = 100L; static final Properties STREAMS_CONFIG = new Properties(); @@ -137,7 +137,7 @@ public static void setupConfigsAndUtils() { PRODUCER_CONFIG.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - RESULT_CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, APP_ID + "-result-consumer"); + RESULT_CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, APPID + "-result-consumer"); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java index bd6abc31e735a..3cf3fe188d0ec 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java @@ -50,7 +50,7 @@ public StreamStreamJoinIntegrationTest(boolean cacheEnabled) { public void prepareTopology() throws InterruptedException { super.prepareEnvironment(); - APP_ID = "stream-stream-join-integration-test"; + APPID = "stream-stream-join-integration-test"; builder = new StreamsBuilder(); leftStream = builder.stream(INPUT_TOPIC_LEFT); @@ -59,7 +59,7 @@ public void prepareTopology() throws InterruptedException { @Test public void testInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner"); final List> expectedResult = Arrays.asList( null, @@ -86,7 +86,7 @@ public void testInner() throws Exception { @Test public void testInnerRepartitioned() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-repartitioned"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-repartitioned"); final List> expectedResult = Arrays.asList( null, @@ -116,7 +116,7 @@ public void testInnerRepartitioned() throws Exception { @Test public void testLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-left"); final List> expectedResult = Arrays.asList( null, @@ -143,7 +143,7 @@ public void testLeft() throws Exception { @Test public void testLeftRepartitioned() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left-repartitioned"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-left-repartitioned"); final List> expectedResult = Arrays.asList( null, @@ -173,7 +173,7 @@ public void testLeftRepartitioned() throws Exception { @Test public void testOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-outer"); final List> expectedResult = Arrays.asList( null, @@ -200,7 +200,7 @@ public void testOuter() throws Exception { @Test public void testOuterRepartitioned() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-outer"); final List> expectedResult = Arrays.asList( null, @@ -230,7 +230,7 @@ public void testOuterRepartitioned() throws Exception { @Test public void testMultiInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-multi-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-multi-inner"); final List> expectedResult = Arrays.asList( null, diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java index 2dd86e2970639..b47fbfa57bb78 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -52,7 +52,7 @@ public TableTableJoinIntegrationTest(boolean cacheEnabled) { public void prepareTopology() throws InterruptedException { super.prepareEnvironment(); - APP_ID = "table-table-join-integration-test"; + APPID = "table-table-join-integration-test"; builder = new StreamsBuilder(); leftTable = builder.table(INPUT_TOPIC_LEFT); @@ -61,7 +61,7 @@ public void prepareTopology() throws InterruptedException { final private String expectedFinalJoinResult = "D-d"; final private String expectedFinalMultiJoinResult = "D-d-d"; - final private String storeName = APP_ID + "-store"; + final private String storeName = APPID + "-store"; private Materialized> materialized = Materialized.>as(storeName) .withKeySerde(Serdes.Long()) @@ -92,7 +92,7 @@ public void apply(final Long key, final String value) { @Test public void testInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); @@ -123,7 +123,7 @@ public void testInner() throws Exception { @Test public void testLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-left"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); @@ -154,7 +154,7 @@ public void testLeft() throws Exception { @Test public void testOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-outer"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); @@ -185,7 +185,7 @@ public void testOuter() throws Exception { @Test public void testInnerInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-inner"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) @@ -223,7 +223,7 @@ public void testInnerInner() throws Exception { @Test public void testInnerLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-left"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) @@ -262,7 +262,7 @@ public void testInnerLeft() throws Exception { @Test public void testInnerOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-outer"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) @@ -300,7 +300,7 @@ public void testInnerOuter() throws Exception { @Test public void testLeftInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-inner"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner) @@ -339,7 +339,7 @@ public void testLeftInner() throws Exception { @Test public void testLeftLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-left"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner) @@ -378,7 +378,7 @@ public void testLeftLeft() throws Exception { @Test public void testLeftOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-outer"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner) @@ -416,7 +416,7 @@ public void testLeftOuter() throws Exception { @Test public void testOuterInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-inner"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner) @@ -455,7 +455,7 @@ public void testOuterInner() throws Exception { @Test public void testOuterLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-left"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner) @@ -494,7 +494,7 @@ public void testOuterLeft() throws Exception { @Test public void testOuterOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APP_ID + "-inner-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-outer"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner) From b89cc990c4a90b709f3ac7e35ff3ce20d71b6153 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 11:03:13 -0800 Subject: [PATCH 13/16] fix style again.. --- .../AbstractJoinIntegrationTest.java | 4 +-- .../StreamStreamJoinIntegrationTest.java | 16 +++++------ .../TableTableJoinIntegrationTest.java | 28 +++++++++---------- 3 files changed, 24 insertions(+), 24 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index 594a7cd272c60..7559ebb02f61d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -78,7 +78,7 @@ public static Collection data() { return values; } - static String APPID; + static String appID; private static final Long COMMIT_INTERVAL = 100L; static final Properties STREAMS_CONFIG = new Properties(); @@ -137,7 +137,7 @@ public static void setupConfigsAndUtils() { PRODUCER_CONFIG.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); - RESULT_CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, APPID + "-result-consumer"); + RESULT_CONSUMER_CONFIG.put(ConsumerConfig.GROUP_ID_CONFIG, appID + "-result-consumer"); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); RESULT_CONSUMER_CONFIG.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java index 3cf3fe188d0ec..571dc058d8cad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/StreamStreamJoinIntegrationTest.java @@ -50,7 +50,7 @@ public StreamStreamJoinIntegrationTest(boolean cacheEnabled) { public void prepareTopology() throws InterruptedException { super.prepareEnvironment(); - APPID = "stream-stream-join-integration-test"; + appID = "stream-stream-join-integration-test"; builder = new StreamsBuilder(); leftStream = builder.stream(INPUT_TOPIC_LEFT); @@ -59,7 +59,7 @@ public void prepareTopology() throws InterruptedException { @Test public void testInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner"); final List> expectedResult = Arrays.asList( null, @@ -86,7 +86,7 @@ public void testInner() throws Exception { @Test public void testInnerRepartitioned() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-repartitioned"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-repartitioned"); final List> expectedResult = Arrays.asList( null, @@ -116,7 +116,7 @@ public void testInnerRepartitioned() throws Exception { @Test public void testLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-left"); final List> expectedResult = Arrays.asList( null, @@ -143,7 +143,7 @@ public void testLeft() throws Exception { @Test public void testLeftRepartitioned() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-left-repartitioned"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-left-repartitioned"); final List> expectedResult = Arrays.asList( null, @@ -173,7 +173,7 @@ public void testLeftRepartitioned() throws Exception { @Test public void testOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-outer"); final List> expectedResult = Arrays.asList( null, @@ -200,7 +200,7 @@ public void testOuter() throws Exception { @Test public void testOuterRepartitioned() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-outer"); final List> expectedResult = Arrays.asList( null, @@ -230,7 +230,7 @@ public void testOuterRepartitioned() throws Exception { @Test public void testMultiInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-multi-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-multi-inner"); final List> expectedResult = Arrays.asList( null, diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java index b47fbfa57bb78..54d997bf9a22c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -52,7 +52,7 @@ public TableTableJoinIntegrationTest(boolean cacheEnabled) { public void prepareTopology() throws InterruptedException { super.prepareEnvironment(); - APPID = "table-table-join-integration-test"; + appID = "table-table-join-integration-test"; builder = new StreamsBuilder(); leftTable = builder.table(INPUT_TOPIC_LEFT); @@ -61,7 +61,7 @@ public void prepareTopology() throws InterruptedException { final private String expectedFinalJoinResult = "D-d"; final private String expectedFinalMultiJoinResult = "D-d-d"; - final private String storeName = APPID + "-store"; + final private String storeName = appID + "-store"; private Materialized> materialized = Materialized.>as(storeName) .withKeySerde(Serdes.Long()) @@ -92,7 +92,7 @@ public void apply(final Long key, final String value) { @Test public void testInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); @@ -123,7 +123,7 @@ public void testInner() throws Exception { @Test public void testLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-left"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); @@ -154,7 +154,7 @@ public void testLeft() throws Exception { @Test public void testOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-outer"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner, materialized).toStream().peek(new CountingPeek(false)).to(OUTPUT_TOPIC); @@ -185,7 +185,7 @@ public void testOuter() throws Exception { @Test public void testInnerInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-inner"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) @@ -223,7 +223,7 @@ public void testInnerInner() throws Exception { @Test public void testInnerLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-left"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) @@ -262,7 +262,7 @@ public void testInnerLeft() throws Exception { @Test public void testInnerOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-outer"); if (cacheEnabled) { leftTable.join(rightTable, valueJoiner) @@ -300,7 +300,7 @@ public void testInnerOuter() throws Exception { @Test public void testLeftInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-inner"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner) @@ -339,7 +339,7 @@ public void testLeftInner() throws Exception { @Test public void testLeftLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-left"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner) @@ -378,7 +378,7 @@ public void testLeftLeft() throws Exception { @Test public void testLeftOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-outer"); if (cacheEnabled) { leftTable.leftJoin(rightTable, valueJoiner) @@ -416,7 +416,7 @@ public void testLeftOuter() throws Exception { @Test public void testOuterInner() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-inner"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-inner"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner) @@ -455,7 +455,7 @@ public void testOuterInner() throws Exception { @Test public void testOuterLeft() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-left"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-left"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner) @@ -494,7 +494,7 @@ public void testOuterLeft() throws Exception { @Test public void testOuterOuter() throws Exception { - STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, APPID + "-inner-outer"); + STREAMS_CONFIG.put(StreamsConfig.APPLICATION_ID_CONFIG, appID + "-inner-outer"); if (cacheEnabled) { leftTable.outerJoin(rightTable, valueJoiner) From 3728b8f119e2a94972cc412f65dee183556040dd Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 11:15:57 -0800 Subject: [PATCH 14/16] add a FIXME comment --- .../streams/integration/TableTableJoinIntegrationTest.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java index 54d997bf9a22c..f3eceb097356c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/TableTableJoinIntegrationTest.java @@ -195,6 +195,8 @@ public void testInnerInner() throws Exception { .to(OUTPUT_TOPIC); runTest(expectedFinalMultiJoinResult, storeName); } else { + // FIXME: the duplicate below for all the multi-joins + // are due to KAFKA-6443, should be updated once it is fixed. List> expectedResult = Arrays.asList( null, null, From f660fc5e4c84671b7f2aae6b5281d4d6bbe0c642 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 11 Jan 2018 17:17:18 -0800 Subject: [PATCH 15/16] github comments --- .../kafka/streams/state/internals/CachingKeyValueStore.java | 5 ++--- .../streams/integration/AbstractJoinIntegrationTest.java | 5 +---- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java index 735b14850868f..f9ab3f1244959 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/CachingKeyValueStore.java @@ -92,13 +92,12 @@ private void putAndMaybeForward(final ThreadCache.DirtyEntry entry, final Intern context.setRecordContext(entry.recordContext()); if (flushListener != null) { final V oldValue = sendOldValues ? serdes.valueFrom(underlying.get(entry.key())) : null; - underlying.put(entry.key(), entry.newValue()); - flushListener.apply(serdes.keyFrom(entry.key().get()), serdes.valueFrom(entry.newValue()), oldValue); - + } else { + underlying.put(entry.key(), entry.newValue()); } } finally { context.setRecordContext(current); diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index 7559ebb02f61d..59144869a361a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -70,7 +70,7 @@ public abstract class AbstractJoinIntegrationTest { @Rule public final TemporaryFolder testFolder = new TemporaryFolder(TestUtils.tempDirectory()); - @Parameterized.Parameters + @Parameterized.Parameters(name = "caching enabled = {0}") public static Collection data() { List values = new ArrayList<>(); for (boolean cacheEnabled : Arrays.asList(true, false)) @@ -192,8 +192,6 @@ void runTest(final List> expectedResult) throws Exception { void runTest(final List> expectedResult, final String storeName) throws Exception { assert expectedResult.size() == input.size(); - System.out.println(builder.build().describe()); - IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); @@ -250,7 +248,6 @@ void runTest(final String expectedFinalResult, final String storeName) throws Ex TestUtils.waitForCondition(new TestCondition() { @Override public boolean conditionMet() { - System.out.println("RESULT: " + finalResultReached.get()); return finalResultReached.get(); } }, "Never received expected final result."); From c2dde66b97859165d466d21376da02c8141d7060 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 12 Jan 2018 14:21:06 -0800 Subject: [PATCH 16/16] github comments --- .../AbstractJoinIntegrationTest.java | 20 ++++++++++--------- 1 file changed, 11 insertions(+), 9 deletions(-) diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java index 59144869a361a..16d2611a8a37d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java @@ -214,9 +214,9 @@ void runTest(final List> expectedResult, final String storeName) th } } - if (storeName != null) + if (storeName != null) { checkQueryableStore(storeName, expectedFinalResult); - + } } finally { streams.close(); } @@ -254,9 +254,9 @@ public boolean conditionMet() { checkResult(OUTPUT_TOPIC, expectedFinalResult, numRecordsExpected); - if (storeName != null) + if (storeName != null) { checkQueryableStore(storeName, expectedFinalResult); - + } } finally { streams.close(); } @@ -271,11 +271,13 @@ private void checkQueryableStore(final String queryableName, final String expect final KeyValueIterator all = store.all(); final KeyValue onlyEntry = all.next(); - assertThat(onlyEntry.key, is(anyUniqueKey)); - assertThat(onlyEntry.value, is(expectedFinalResult)); - assertThat(all.hasNext(), is(false)); - - all.close(); + try { + assertThat(onlyEntry.key, is(anyUniqueKey)); + assertThat(onlyEntry.value, is(expectedFinalResult)); + assertThat(all.hasNext(), is(false)); + } finally { + all.close(); + } } private final class Input {