/** * Copyright 2016 Confluent Inc. * * Licensed 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 io.confluent.examples.streams; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.producer.ProducerConfig; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.test.TestUtils; import org.junit.BeforeClass; import org.junit.ClassRule; import org.junit.Test; import java.util.Arrays; import java.util.List; import java.util.Properties; import io.confluent.examples.streams.kafka.EmbeddedSingleNodeKafkaCluster; import static org.assertj.core.api.Assertions.assertThat; /** * End-to-end integration test that shows how to compute the sum of numbers, based on {@link * SumLambdaExample}. * * Note: This example uses lambda expressions and thus works with Java 8+ only. */ public class SumLambdaIntegrationTest { @ClassRule public static final EmbeddedSingleNodeKafkaCluster CLUSTER = new EmbeddedSingleNodeKafkaCluster(); private static String inputTopic = "inputTopic"; private static String outputTopic = "outputTopic"; @BeforeClass public static void startKafkaCluster() throws Exception { CLUSTER.createTopic(inputTopic); CLUSTER.createTopic(outputTopic); } @Test public void shouldSumEvenNumbers() throws Exception { List<Integer> inputValues = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); List<Integer> expectedValues = Arrays.asList(30); // // Step 1: Configure and start the processor topology. // KStreamBuilder builder = new KStreamBuilder(); Properties streamsConfiguration = new Properties(); streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "sum-lambda-integration-test"); streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); streamsConfiguration.put(StreamsConfig.DEFAULT_KEY_SERDE_CLASS_CONFIG, Serdes.Integer().getClass().getName()); streamsConfiguration.put(StreamsConfig.DEFAULT_VALUE_SERDE_CLASS_CONFIG, Serdes.Integer().getClass().getName()); // The commit interval for flushing records to state stores and downstream must be lower than // this integration test's timeout (30 secs) to ensure we observe the expected processing results. streamsConfiguration.put(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG, 10 * 1000); streamsConfiguration.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); // Use a temporary directory for storing state, which will be automatically removed after the test. streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, TestUtils.tempDirectory().getAbsolutePath()); KStream<Integer, Integer> input = builder.stream(inputTopic); KTable<Integer, Integer> sumOfOddNumbers = input .filter((k, v) -> v % 2 == 0) .selectKey((k, v) -> 1) // no need to specify explicit serdes because the resulting key and value types match our default serde settings .groupByKey() .reduce((v1, v2) -> v1 + v2, "sum"); sumOfOddNumbers.to(outputTopic); KafkaStreams streams = new KafkaStreams(builder, streamsConfiguration); streams.start(); // // Step 2: Produce some input data to the input topic. // 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, IntegerSerializer.class); producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); IntegrationTestUtils.produceValuesSynchronously(inputTopic, inputValues, producerConfig); // // Step 3: Verify the application's output data. // Properties consumerConfig = new Properties(); consumerConfig.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, CLUSTER.bootstrapServers()); consumerConfig.put(ConsumerConfig.GROUP_ID_CONFIG, "sum-lambda-integration-test-standard-consumer"); consumerConfig.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); consumerConfig.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); consumerConfig.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); List<String> actualValues = IntegrationTestUtils.waitUntilMinValuesRecordsReceived(consumerConfig, outputTopic, expectedValues.size()); streams.close(); assertThat(actualValues).isEqualTo(expectedValues); } }