-
Notifications
You must be signed in to change notification settings - Fork 15.1k
MINOR: Improve Join integration test coverage, PART I #4331
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Closed
guozhangwang
wants to merge
23
commits into
apache:trunk
from
guozhangwang:KMinor-join-integration-tests
Closed
Changes from all commits
Commits
Show all changes
23 commits
Select commit
Hold shift + click to select a range
c1e00d0
ingested stream stream join
guozhangwang 71f2625
Merge branch 'trunk' of https://git-wip-us.apache.org/repos/asf/kafka…
guozhangwang 4d95bdb
create a base abstract join class
guozhangwang defd7ed
Add more tests
guozhangwang 2a55c66
Merge branch 'trunk' of https://git-wip-us.apache.org/repos/asf/kafka…
guozhangwang 73f58fb
consider using mock time
guozhangwang d210de3
Merge branch 'trunk' of https://github.com/apache/kafka into KMinor-j…
guozhangwang ae2ba11
fix a bug in flushing ordering
guozhangwang 4d5e80f
Merge branch 'trunk' of https://github.com/apache/kafka into KMinor-j…
guozhangwang eb5f995
rebase from trunk
guozhangwang d495386
Add more tests to table table join
guozhangwang a93ee09
Merge branch 'trunk' of https://github.com/apache/kafka into KMinor-j…
guozhangwang a7bb2e8
minor fixes
guozhangwang 33ef938
remove debugging prints
guozhangwang 2c33f17
remove old table table join integration test
guozhangwang e01a072
rename functions
guozhangwang 1ee24db
more renames and minor fixes
guozhangwang 6285217
fix style
guozhangwang b89cc99
fix style again..
guozhangwang 3728b8f
add a FIXME comment
guozhangwang f660fc5
github comments
guozhangwang 3d2499c
Merge branch 'trunk' of https://github.com/apache/kafka into KMinor-j…
guozhangwang c2dde66
github comments
guozhangwang File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
292 changes: 292 additions & 0 deletions
292
streams/src/test/java/org/apache/kafka/streams/integration/AbstractJoinIntegrationTest.java
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,292 @@ | ||
| /* | ||
| * 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.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; | ||
| import org.junit.After; | ||
| import org.junit.BeforeClass; | ||
| import org.junit.ClassRule; | ||
| import org.junit.Rule; | ||
| 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.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; | ||
|
|
||
| /** | ||
| * 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(name = "caching enabled = {0}") | ||
| public static Collection<Object[]> data() { | ||
| List<Object[]> values = new ArrayList<>(); | ||
| for (boolean cacheEnabled : Arrays.asList(true, false)) | ||
| values.add(new Object[] {cacheEnabled}); | ||
| return values; | ||
| } | ||
|
|
||
| static String appID; | ||
|
|
||
| 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"; | ||
| 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<Long, String> producer; | ||
| private KafkaStreams streams; | ||
|
|
||
| StreamsBuilder builder; | ||
| int numRecordsExpected = 0; | ||
| AtomicBoolean finalResultReached = new AtomicBoolean(false); | ||
|
|
||
| private final List<Input<String>> 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<String, String, String> valueJoiner = new ValueJoiner<String, String, String>() { | ||
| @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, 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); | ||
|
|
||
| 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, COMMIT_INTERVAL); | ||
| } | ||
|
|
||
| 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<String> expectedResult) throws InterruptedException { | ||
| final List<String> 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 { | ||
| final List<String> 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 | ||
| */ | ||
| void runTest(final List<List<String>> 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<List<String>> expectedResult, final String storeName) throws Exception { | ||
| assert expectedResult.size() == input.size(); | ||
|
|
||
| IntegrationTestUtils.purgeLocalStreamsState(STREAMS_CONFIG); | ||
| streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); | ||
|
|
||
| String expectedFinalResult = null; | ||
|
|
||
| try { | ||
| streams.start(); | ||
|
|
||
| long ts = System.currentTimeMillis(); | ||
|
|
||
| final Iterator<List<String>> resultIterator = expectedResult.iterator(); | ||
| for (final Input<String> singleInput : input) { | ||
| producer.send(new ProducerRecord<>(singleInput.topic, null, ++ts, singleInput.record.key, singleInput.record.value)).get(); | ||
|
|
||
| List<String> expected = resultIterator.next(); | ||
|
|
||
| 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. | ||
| */ | ||
| 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); | ||
| streams = new KafkaStreams(builder.build(), new StreamsConfig(STREAMS_CONFIG)); | ||
|
|
||
| try { | ||
| streams.start(); | ||
|
|
||
| long ts = System.currentTimeMillis(); | ||
|
|
||
| for (final Input<String> 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() { | ||
| return finalResultReached.get(); | ||
| } | ||
| }, "Never received expected final result."); | ||
|
|
||
| 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<Long, String> store = streams.store(queryableName, QueryableStoreTypes.<Long, String>keyValueStore()); | ||
|
|
||
| final KeyValueIterator<Long, String> all = store.all(); | ||
| final KeyValue<Long, String> onlyEntry = all.next(); | ||
|
|
||
| try { | ||
| assertThat(onlyEntry.key, is(anyUniqueKey)); | ||
| assertThat(onlyEntry.value, is(expectedFinalResult)); | ||
| assertThat(all.hasNext(), is(false)); | ||
| } finally { | ||
| all.close(); | ||
| } | ||
| } | ||
|
|
||
| private final class Input<V> { | ||
| String topic; | ||
| KeyValue<Long, V> record; | ||
|
|
||
| Input(final String topic, final V value) { | ||
| this.topic = topic; | ||
| record = KeyValue.pair(anyUniqueKey, value); | ||
| } | ||
| } | ||
| } |
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this actually needed? If a downstream processor tries to get this value won't it get it from the cache? i.e., i don't think an evicted entry is removed from the cache until after flush has finished
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Actually it will be removed, i.e. when
listener.apply(entries);is called the entry is no longer in the cache anymore.What I observed originally is an issue when caching is turned off: note that with that case we still go through this code path (which should be optimized away anyways in the future I think), when you call a
puton the store, it will immediately trigger flush and hence be processed downstream while it is not put into the underlying and also not in the cache any more.