diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index dd37df4cc0765..d8a111edb1481 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -56,6 +56,11 @@ do CLASSPATH=$CLASSPATH:$file done +for file in $base_dir/stream/build/libs/kafka-streams*.jar; +do + CLASSPATH=$CLASSPATH:$file +done + for file in $base_dir/tools/build/libs/kafka-tools*.jar; do CLASSPATH=$CLASSPATH:$file diff --git a/build.gradle b/build.gradle index fecc3eb3b6918..02b1db538e773 100644 --- a/build.gradle +++ b/build.gradle @@ -215,25 +215,22 @@ for ( sv in ['2_10_5', '2_11_7'] ) { } def copycatPkgs = ['copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file'] -def pkgs = ['clients', 'examples', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'log4j-appender', 'tools'] + copycatPkgs +def pkgs = ['clients', 'examples', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'log4j-appender', 'tools', 'streams'] + copycatPkgs tasks.create(name: "jarCopycat", dependsOn: copycatPkgs.collect { it + ":jar" }) {} -tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7'] + pkgs.collect { it + ":jar" }) { -} +tasks.create(name: "jarAll", dependsOn: ['jar_core_2_10_5', 'jar_core_2_11_7'] + pkgs.collect { it + ":jar" }) { } tasks.create(name: "srcJarAll", dependsOn: ['srcJar_2_10_5', 'srcJar_2_11_7'] + pkgs.collect { it + ":srcJar" }) { } tasks.create(name: "docsJarAll", dependsOn: ['docsJar_2_10_5', 'docsJar_2_11_7'] + pkgs.collect { it + ":docsJar" }) { } tasks.create(name: "testCopycat", dependsOn: copycatPkgs.collect { it + ":test" }) {} -tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7'] + pkgs.collect { it + ":test" }) { -} +tasks.create(name: "testAll", dependsOn: ['test_core_2_10_5', 'test_core_2_11_7'] + pkgs.collect { it + ":test" }) { } tasks.create(name: "releaseTarGzAll", dependsOn: ['releaseTarGz_2_10_5', 'releaseTarGz_2_11_7']) { } -tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7'] + pkgs.collect { it + ":uploadArchives" }) { -} +tasks.create(name: "uploadArchivesAll", dependsOn: ['uploadCoreArchives_2_10_5', 'uploadCoreArchives_2_11_7'] + pkgs.collect { it + ":uploadArchives" }) { } project(':core') { println "Building project 'core' with Scala version $scalaVersion" @@ -518,6 +515,71 @@ project(':tools') { dependsOn 'copyDependantLibs' } + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + + checkstyle { + configFile = new File(rootDir, "checkstyle/checkstyle.xml") + } + test.dependsOn('checkstyleMain', 'checkstyleTest') +} + +project(':streams') { + apply plugin: 'checkstyle' + archivesBaseName = "kafka-streams" + + dependencies { + compile project(':clients') + compile "$slf4jlog4j" + compile 'org.rocksdb:rocksdbjni:3.10.1' + + testCompile "$junit" + testCompile project(path: ':clients', configuration: 'archives') + } + + task testJar(type: Jar) { + classifier = 'test' + from sourceSets.test.output + } + + test { + testLogging { + events "passed", "skipped", "failed" + exceptionFormat = 'full' + } + } + + javadoc { + include "**/org/apache/kafka/streams/*" + } + + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntime) { + include('slf4j-log4j12*') + } + from (configurations.runtime) { + exclude('kafka-clients*') + } + into "$buildDir/dependant-libs-${scalaVersion}" + } + + jar { + dependsOn 'copyDependantLibs' + } + + artifacts { + archives testJar + } + + configurations { + archives.extendsFrom (testCompile) + } + checkstyle { configFile = new File(rootDir, "checkstyle/checkstyle.xml") } diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index a215ff36e9252..999fd6ca1dde9 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -53,9 +53,13 @@ - - + + + + + + diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index d58c472872ced..7b748eca56f8a 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -90,8 +90,8 @@ - + @@ -111,6 +111,20 @@ + + + + + + + + + + + + + + diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index b9a2d4e2bc565..347a5bc00c78d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -150,11 +150,11 @@ public class ConsumerConfig extends AbstractConfig { /** key.deserializer */ public static final String KEY_DESERIALIZER_CLASS_CONFIG = "key.deserializer"; - private static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; + public static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the Deserializer interface."; /** value.deserializer */ public static final String VALUE_DESERIALIZER_CLASS_CONFIG = "value.deserializer"; - private static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; + public static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the Deserializer interface."; /** connections.max.idle.ms */ public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 2a3c763ae7afb..68f61bff7b0f2 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -550,6 +550,7 @@ private KafkaConsumer(ConsumerConfig config, Deserializer.class); this.keyDeserializer.configure(config.originals(), true); } else { + config.ignore(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG); this.keyDeserializer = keyDeserializer; } if (valueDeserializer == null) { @@ -557,6 +558,7 @@ private KafkaConsumer(ConsumerConfig config, Deserializer.class); this.valueDeserializer.configure(config.originals(), false); } else { + config.ignore(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG); this.valueDeserializer = valueDeserializer; } this.fetcher = new Fetcher(this.client, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index 1f802a8097072..3c0f26114d029 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -46,6 +46,7 @@ public class MockConsumer implements Consumer { private final Map> partitions; private final SubscriptionState subscriptions; private Map>> records; + private Set paused; private boolean closed; private final Map beginningOffsets; private final Map endOffsets; @@ -57,8 +58,9 @@ public class MockConsumer implements Consumer { public MockConsumer(OffsetResetStrategy offsetResetStrategy) { this.subscriptions = new SubscriptionState(offsetResetStrategy); - this.partitions = new HashMap>(); - this.records = new HashMap>>(); + this.partitions = new HashMap<>(); + this.records = new HashMap<>(); + this.paused = new HashSet<>(); this.closed = false; this.beginningOffsets = new HashMap<>(); this.endOffsets = new HashMap<>(); @@ -288,14 +290,18 @@ public void updatePartitions(String topic, List partitions) { @Override public void pause(TopicPartition... partitions) { - for (TopicPartition partition : partitions) + for (TopicPartition partition : partitions) { subscriptions.pause(partition); + paused.add(partition); + } } @Override public void resume(TopicPartition... partitions) { - for (TopicPartition partition : partitions) + for (TopicPartition partition : partitions) { subscriptions.resume(partition); + paused.remove(partition); + } } @Override @@ -332,6 +338,10 @@ public void waitForPollThen(Runnable task, long timeoutMs) { } } + public Set paused() { + return Collections.unmodifiableSet(new HashSet<>(paused)); + } + private void ensureNotClosed() { if (this.closed) throw new IllegalStateException("This consumer has already been closed."); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 804d569498396..3a783ec4164a0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -259,6 +259,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial Serializer.class); this.keySerializer.configure(config.originals(), true); } else { + config.ignore(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG); this.keySerializer = keySerializer; } if (valueSerializer == null) { @@ -266,6 +267,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial Serializer.class); this.valueSerializer.configure(config.originals(), false); } else { + config.ignore(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG); this.valueSerializer = valueSerializer; } config.logUnused(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index 06f00a99a73a2..6969f61a130bc 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -164,11 +164,11 @@ public class ProducerConfig extends AbstractConfig { /** key.serializer */ public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer"; - private static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; + public static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the Serializer interface."; /** value.serializer */ public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer"; - private static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; + public static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the Serializer interface."; /** connections.max.idle.ms */ public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG; diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index 12a19274ffb56..2961e09d7d1f1 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -62,6 +62,10 @@ protected Object get(String key) { return values.get(key); } + public void ignore(String key) { + used.add(key); + } + public Short getShort(String key) { return (Short) get(key); } diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java new file mode 100644 index 0000000000000..37983e4a4c031 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongDeserializer.java @@ -0,0 +1,44 @@ +/** + * 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.common.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.util.Map; + +public class LongDeserializer implements Deserializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public Long deserialize(String topic, byte[] data) { + if (data == null) + return null; + if (data.length != 8) { + throw new SerializationException("Size of data received by LongDeserializer is " + + "not 8"); + } + + long value = 0; + for (byte b : data) { + value <<= 8; + value |= b & 0xFF; + } + return value; + } + + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java new file mode 100644 index 0000000000000..31005299e7a41 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/LongSerializer.java @@ -0,0 +1,42 @@ +/** + * 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.common.serialization; + +import java.util.Map; + +public class LongSerializer implements Serializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public byte[] serialize(String topic, Long data) { + if (data == null) + return null; + + return new byte[] { + (byte) (data >>> 56), + (byte) (data >>> 48), + (byte) (data >>> 40), + (byte) (data >>> 32), + (byte) (data >>> 24), + (byte) (data >>> 16), + (byte) (data >>> 8), + data.byteValue() + }; + } + + public void close() { + // nothing to do + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index c58b74144ba01..fa7c92f08502c 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -25,7 +25,10 @@ import java.nio.MappedByteBuffer; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; +import java.util.HashSet; import java.util.Iterator; +import java.util.List; import java.util.regex.Matcher; import java.util.regex.Pattern; import java.util.Properties; @@ -524,4 +527,42 @@ public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength return existingBuffer; } + /* + * Creates a set + * @param elems the elements + * @param the type of element + * @return Set + */ + public static HashSet mkSet(T... elems) { + return new HashSet<>(Arrays.asList(elems)); + } + + /** + * Recursively delete the given file/directory and any subfiles (if any exist) + * + * @param file The root file at which to begin deleting + */ + public static void delete(File file) { + if (file == null) { + return; + } else if (file.isDirectory()) { + File[] files = file.listFiles(); + if (files != null) { + for (File f : files) + delete(f); + } + file.delete(); + } else { + file.delete(); + } + } + + /** + * Returns an empty collection if this list is null + * @param other + * @return + */ + public static List safe(List other) { + return other == null ? Collections.emptyList() : other; + } } diff --git a/settings.gradle b/settings.gradle index 9c7fea5dce257..357305ba23e8c 100644 --- a/settings.gradle +++ b/settings.gradle @@ -14,5 +14,5 @@ // limitations under the License. apply from: file('scala.gradle') -include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'log4j-appender', - 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file' +include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'streams', 'log4j-appender', + 'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file' \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java new file mode 100644 index 0000000000000..f3a99e0168caa --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreaming.java @@ -0,0 +1,125 @@ +/** + * 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; + +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.internals.StreamThread; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Kafka Streaming allows for performing continuous computation on input coming from one or more input topics and + * sends output to zero or more output topics. + *

+ * This processing is defined by using the {@link TopologyBuilder} class or its superclass KStreamBuilder to specify + * the transformation. + * The {@link KafkaStreaming} instance will be responsible for the lifecycle of these processors. It will instantiate and + * start one or more of these processors to process the Kafka partitions assigned to this particular instance. + *

+ * This streaming instance will co-ordinate with any other instances (whether in this same process, on other processes + * on this machine, or on remote machines). These processes will divide up the work so that all partitions are being + * consumed. If instances are added or die, the corresponding {@link StreamThread} instances will be shutdown or + * started in the appropriate processes to balance processing load. + *

+ * Internally the {@link KafkaStreaming} instance contains a normal {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer} + * and {@link org.apache.kafka.clients.consumer.KafkaConsumer KafkaConsumer} instance that is used for reading input and writing output. + *

+ * A simple example might look like this: + *

+ *    Map<String, Object> props = new HashMap<>();
+ *    props.put("bootstrap.servers", "localhost:4242");
+ *    props.put("key.deserializer", StringDeserializer.class);
+ *    props.put("value.deserializer", StringDeserializer.class);
+ *    props.put("key.serializer", StringSerializer.class);
+ *    props.put("value.serializer", IntegerSerializer.class);
+ *    props.put("timestamp.extractor", MyTimestampExtractor.class);
+ *    StreamingConfig config = new StreamingConfig(props);
+ *
+ *    KStreamBuilder builder = new KStreamBuilder();
+ *    builder.from("topic1").mapValue(value -> value.length()).to("topic2");
+ *
+ *    KafkaStreaming streaming = new KafkaStreaming(builder, config);
+ *    streaming.start();
+ * 
+ * + */ +public class KafkaStreaming { + + private static final Logger log = LoggerFactory.getLogger(KafkaStreaming.class); + + // Container States + private static final int CREATED = 0; + private static final int RUNNING = 1; + private static final int STOPPED = 2; + private int state = CREATED; + + private final StreamThread[] threads; + + public KafkaStreaming(TopologyBuilder builder, StreamingConfig config) throws Exception { + this.threads = new StreamThread[config.getInt(StreamingConfig.NUM_STREAM_THREADS_CONFIG)]; + for (int i = 0; i < this.threads.length; i++) { + this.threads[i] = new StreamThread(builder, config); + } + } + + /** + * Start the stream process by starting all its threads + */ + public synchronized void start() { + log.debug("Starting Kafka Stream process"); + + if (state == CREATED) { + for (StreamThread thread : threads) + thread.start(); + + state = RUNNING; + + log.info("Started Kafka Stream process"); + } else { + throw new IllegalStateException("This process was already started."); + } + } + + /** + * Shutdown this stream process by signaling the threads to stop, + * wait for them to join and clean up the process instance. + */ + public synchronized void close() { + log.debug("Stopping Kafka Stream process"); + + if (state == RUNNING) { + // signal the threads to stop and wait + for (StreamThread thread : threads) + thread.close(); + + for (StreamThread thread : threads) { + try { + thread.join(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + + state = STOPPED; + + log.info("Stopped Kafka Stream process"); + } else { + throw new IllegalStateException("This process has not started yet."); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java new file mode 100644 index 0000000000000..dce69b6000cfd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/StreamingConfig.java @@ -0,0 +1,201 @@ +/** + * 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; + +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.config.AbstractConfig; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; + +import java.util.Map; + +public class StreamingConfig extends AbstractConfig { + + private static final ConfigDef CONFIG; + + /** state.dir */ + public static final String STATE_DIR_CONFIG = "state.dir"; + private static final String STATE_DIR_DOC = "Directory location for state store."; + + /** commit.interval.ms */ + public static final String COMMIT_INTERVAL_MS_CONFIG = "commit.interval.ms"; + private static final String COMMIT_INTERVAL_MS_DOC = "The frequency with which to save the position of the processor."; + + /** poll.ms */ + public static final String POLL_MS_CONFIG = "poll.ms"; + private static final String POLL_MS_DOC = "The amount of time in milliseconds to block waiting for input."; + + /** num.stream.threads */ + public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads"; + private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing."; + + /** buffered.records.per.partition */ + public static final String BUFFERED_RECORDS_PER_PARTITION_CONFIG = "buffered.records.per.partition"; + private static final String BUFFERED_RECORDS_PER_PARTITION_DOC = "The maximum number of records to buffer per partition."; + + /** state.cleanup.delay */ + public static final String STATE_CLEANUP_DELAY_MS_CONFIG = "state.cleanup.delay.ms"; + private static final String STATE_CLEANUP_DELAY_MS_DOC = "The amount of time in milliseconds to wait before deleting state when a partition has migrated."; + + /** total.records.to.process */ + public static final String TOTAL_RECORDS_TO_PROCESS = "total.records.to.process"; + private static final String TOTAL_RECORDS_TO_DOC = "Exit after processing this many records."; + + /** window.time.ms */ + public static final String WINDOW_TIME_MS_CONFIG = "window.time.ms"; + private static final String WINDOW_TIME_MS_DOC = "Setting this to a non-negative value will cause the processor to get called " + + "with this frequency even if there is no message."; + + /** timestamp.extractor */ + public static final String TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "timestamp.extractor"; + private static final String TIMESTAMP_EXTRACTOR_CLASS_DOC = "Timestamp extractor class that implements the TimestampExtractor interface."; + + /** client.id */ + public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; + + /** key.serializer */ + public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; + + /** value.serializer */ + public static final String VALUE_SERIALIZER_CLASS_CONFIG = ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; + + /** key.deserializer */ + public static final String KEY_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; + + /** value.deserializer */ + public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + + /** + * bootstrap.servers + */ + public static final String BOOTSTRAP_SERVERS_CONFIG = CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG; + + private static final String SYSTEM_TEMP_DIRECTORY = System.getProperty("java.io.tmpdir"); + + static { + CONFIG = new ConfigDef().define(CLIENT_ID_CONFIG, + Type.STRING, + "", + Importance.MEDIUM, + CommonClientConfigs.CLIENT_ID_DOC) + .define(STATE_DIR_CONFIG, + Type.STRING, + SYSTEM_TEMP_DIRECTORY, + Importance.MEDIUM, + STATE_DIR_DOC) + .define(COMMIT_INTERVAL_MS_CONFIG, + Type.LONG, + 30000, + Importance.HIGH, + COMMIT_INTERVAL_MS_DOC) + .define(POLL_MS_CONFIG, + Type.LONG, + 100, + Importance.LOW, + POLL_MS_DOC) + .define(NUM_STREAM_THREADS_CONFIG, + Type.INT, + 1, + Importance.LOW, + NUM_STREAM_THREADS_DOC) + .define(BUFFERED_RECORDS_PER_PARTITION_CONFIG, + Type.INT, + 1000, + Importance.LOW, + BUFFERED_RECORDS_PER_PARTITION_DOC) + .define(STATE_CLEANUP_DELAY_MS_CONFIG, + Type.LONG, + 60000, + Importance.LOW, + STATE_CLEANUP_DELAY_MS_DOC) + .define(TOTAL_RECORDS_TO_PROCESS, + Type.LONG, + -1L, + Importance.LOW, + TOTAL_RECORDS_TO_DOC) + .define(WINDOW_TIME_MS_CONFIG, + Type.LONG, + -1L, + Importance.MEDIUM, + WINDOW_TIME_MS_DOC) + .define(KEY_SERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ProducerConfig.KEY_SERIALIZER_CLASS_DOC) + .define(VALUE_SERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ProducerConfig.VALUE_SERIALIZER_CLASS_DOC) + .define(KEY_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ConsumerConfig.KEY_DESERIALIZER_CLASS_DOC) + .define(VALUE_DESERIALIZER_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + ConsumerConfig.VALUE_DESERIALIZER_CLASS_DOC) + .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, + Type.CLASS, + Importance.HIGH, + TIMESTAMP_EXTRACTOR_CLASS_DOC) + .define(BOOTSTRAP_SERVERS_CONFIG, + Type.STRING, + Importance.HIGH, + CommonClientConfigs.BOOSTRAP_SERVERS_DOC); + } + + public StreamingConfig(Map props) { + super(CONFIG, props); + } + + public Map getConsumerConfigs() { + Map props = this.originals(); + + // set consumer default property values + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, "range"); + + // remove properties that are not required for consumers + props.remove(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + + return props; + } + + public Map getProducerConfigs() { + Map props = this.originals(); + + // set producer default property values + props.put(ProducerConfig.LINGER_MS_CONFIG, "100"); + + // remove properties that are not required for producers + props.remove(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG); + props.remove(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + + return props; + } + + public static void main(String[] args) { + System.out.println(CONFIG.toHtmlTable()); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java new file mode 100644 index 0000000000000..feb4ee7d41f2e --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/KStreamJob.java @@ -0,0 +1,84 @@ +/** + * 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.examples; + +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.KafkaStreaming; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Predicate; + +import java.util.Properties; + +public class KStreamJob { + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamingConfig.CLIENT_ID_CONFIG, "Example-KStream-Job"); + props.put(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + props.put(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); + props.put(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); + StreamingConfig config = new StreamingConfig(props); + + KStreamBuilder builder = new KStreamBuilder(); + + KStream stream1 = builder.from("topic1"); + + KStream stream2 = + stream1.map(new KeyValueMapper>() { + @Override + public KeyValue apply(String key, String value) { + return new KeyValue<>(key, new Integer(value)); + } + }).filter(new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + }); + + KStream[] streams = stream2.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); + + streams[0].to("topic2"); + streams[1].to("topic3"); + + KafkaStreaming kstream = new KafkaStreaming(builder, config); + kstream.start(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java new file mode 100644 index 0000000000000..0b3aba8716aed --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/ProcessorJob.java @@ -0,0 +1,112 @@ +/** + * 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.examples; + +import org.apache.kafka.common.serialization.IntegerSerializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.KafkaStreaming; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.state.Entry; +import org.apache.kafka.streams.state.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.Properties; + +public class ProcessorJob { + + private static class MyProcessorDef implements ProcessorDef { + + @Override + public Processor instance() { + return new Processor() { + private ProcessorContext context; + private KeyValueStore kvStore; + + @Override + public void init(ProcessorContext context) { + this.context = context; + this.context.schedule(1000); + this.kvStore = new InMemoryKeyValueStore<>("local-state", context); + } + + @Override + public void process(String key, String value) { + Integer oldValue = this.kvStore.get(key); + Integer newValue = Integer.parseInt(value); + if (oldValue == null) { + this.kvStore.put(key, newValue); + } else { + this.kvStore.put(key, oldValue + newValue); + } + + context.commit(); + } + + @Override + public void punctuate(long timestamp) { + KeyValueIterator iter = this.kvStore.all(); + + while (iter.hasNext()) { + Entry entry = iter.next(); + + System.out.println("[" + entry.key() + ", " + entry.value() + "]"); + + context.forward(entry.key(), entry.value()); + } + + iter.close(); + } + + @Override + public void close() { + this.kvStore.close(); + } + }; + } + } + + public static void main(String[] args) throws Exception { + Properties props = new Properties(); + props.put(StreamingConfig.CLIENT_ID_CONFIG, "Example-Processor-Job"); + props.put(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); + props.put(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + props.put(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); + props.put(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); + props.put(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); + StreamingConfig config = new StreamingConfig(props); + + TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("SOURCE", new StringDeserializer(), new StringDeserializer(), "topic-source"); + + builder.addProcessor("PROCESS", new MyProcessorDef(), "SOURCE"); + + builder.addSink("SINK", "topic-sink", new StringSerializer(), new IntegerSerializer(), "PROCESS"); + + KafkaStreaming streaming = new KafkaStreaming(builder, config); + streaming.start(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java new file mode 100644 index 0000000000000..26281d69d0293 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/examples/WallclockTimestampExtractor.java @@ -0,0 +1,28 @@ +/** + * 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.examples; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; + +public class WallclockTimestampExtractor implements TimestampExtractor { + @Override + public long extract(ConsumerRecord record) { + return System.currentTimeMillis(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java new file mode 100644 index 0000000000000..7f101ab48af91 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -0,0 +1,156 @@ +/** + * 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.kstream; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.ProcessorDef; + +/** + * KStream is an abstraction of a stream of key-value pairs. + */ +public interface KStream { + + /** + * Creates a new stream consists of all elements of this stream which satisfy a predicate + * + * @param predicate the instance of Predicate + * @return KStream + */ + KStream filter(Predicate predicate); + + /** + * Creates a new stream consists all elements of this stream which do not satisfy a predicate + * + * @param predicate the instance of Predicate + * @return KStream + */ + KStream filterOut(Predicate predicate); + + /** + * Creates a new stream by transforming key-value pairs by a mapper to all elements of this stream + * + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream map(KeyValueMapper> mapper); + + /** + * Creates a new stream by transforming values by a mapper to all values of this stream + * + * @param mapper the instance of ValueMapper + * @param the value type of the new stream + * @return KStream + */ + KStream mapValues(ValueMapper mapper); + + /** + * Creates a new stream by applying a mapper to all elements of this stream and using the values in the resulting Iterable + * + * @param mapper the instance of KeyValueMapper + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream flatMap(KeyValueMapper>> mapper); + + /** + * Creates a new stream by applying a mapper to all values of this stream and using the values in the resulting Iterable + * + * @param processor the instance of Processor + * @param the value type of the new stream + * @return KStream + */ + KStream flatMapValues(ValueMapper> processor); + + /** + * Creates a new windowed stream using a specified window instance. + * + * @param windowDef the instance of Window + * @return KStream + */ + KStreamWindowed with(WindowDef windowDef); + + /** + * Creates an array of streams from this stream. Each stream in the array coresponds to a predicate in + * supplied predicates in the same order. Predicates are evaluated in order. An element is streamed to + * a corresponding stream for the first predicate is evaluated true. + * An element will be dropped if none of the predicates evaluate true. + * + * @param predicates Instances of Predicate + * @return KStream + */ + KStream[] branch(Predicate... predicates); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * This is equivalent to calling to(topic) and from(topic). + * + * @param topic the topic name + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream through(String topic); + + /** + * Sends key-value to a topic, also creates a new stream from the topic. + * This is equivalent to calling to(topic) and from(topic). + * + * @param topic the topic name + * @param keySerializer key serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + * @param valSerializer value serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + * @param keyDeserializer key deserializer used to create the new KStream, + * if not specified the default deserializer defined in the configs will be used + * @param valDeserializer value deserializer used to create the new KStream, + * if not specified the default deserializer defined in the configs will be used + * @param the key type of the new stream + * @param the value type of the new stream + * @return KStream + */ + KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + + /** + * Sends key-value to a topic using default serializers specified in the config. + * + * @param topic the topic name + */ + void to(String topic); + + /** + * Sends key-value to a topic. + * + * @param topic the topic name + * @param keySerializer key serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + * @param valSerializer value serializer used to send key-value pairs, + * if not specified the default serializer defined in the configs will be used + */ + void to(String topic, Serializer keySerializer, Serializer valSerializer); + + /** + * Processes all elements in this stream by applying a processor. + * + * @param processorDef the class of ProcessorDef + */ + KStream process(ProcessorDef processorDef); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java new file mode 100644 index 0000000000000..2d4dcc72f8057 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -0,0 +1,65 @@ +/** + * 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.kstream; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.processor.TopologyBuilder; + +/** + * KStreamBuilder is the class to create KStream instances. + */ +public class KStreamBuilder extends TopologyBuilder { + + public KStreamBuilder() { + super(); + } + + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * The default deserializers specified in the config are used. + * + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(String... topics) { + String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(); + + addSource(name, topics); + + return new KStreamImpl<>(this, name); + } + + /** + * Creates a KStream instance for the specified topic. The stream is added to the default synchronization group. + * + * @param keyDeserializer key deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param valDeserializer value deserializer used to read this source KStream, + * if not specified the default deserializer defined in the configs will be used + * @param topics the topic names, if empty default to all the topics in the config + * @return KStream + */ + public KStream from(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement(); + + addSource(name, keyDeserializer, valDeserializer, topics); + + return new KStreamImpl<>(this, name); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java new file mode 100644 index 0000000000000..4d73128839f21 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamWindowed.java @@ -0,0 +1,38 @@ +/** + * 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.kstream; + +/** + * KStreamWindowed is an abstraction of a stream of key-value pairs with a window. + */ +public interface KStreamWindowed extends KStream { + + /** + * Creates a new stream by joining this windowed stream with the other windowed stream. + * Each element arrived from either of the streams is joined with elements in a window of each other. + * The resulting values are computed by applying a joiner. + * + * @param other the other windowed stream + * @param joiner ValueJoiner + * @param the value type of the other stream + * @param the value type of the new stream + * @return KStream + */ + KStream join(KStreamWindowed other, ValueJoiner joiner); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java new file mode 100644 index 0000000000000..f633f6e337532 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValue.java @@ -0,0 +1,34 @@ +/** + * 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.kstream; + +public class KeyValue { + + public final K key; + public final V value; + + public KeyValue(K key, V value) { + this.key = key; + this.value = value; + } + + public static KeyValue pair(K key, V value) { + return new KeyValue<>(key, value); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java new file mode 100644 index 0000000000000..62b07f646c812 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java @@ -0,0 +1,23 @@ +/** + * 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.kstream; + +public interface KeyValueMapper { + + R apply(K key, V value); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java new file mode 100644 index 0000000000000..9cdb3bc6caa56 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java @@ -0,0 +1,24 @@ +/** + * 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.kstream; + +public interface Predicate { + + boolean apply(K key, V value); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java new file mode 100644 index 0000000000000..cc03541796cd0 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/SlidingWindowDef.java @@ -0,0 +1,265 @@ +/** + * 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.kstream; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.ByteArraySerializer; +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.Serializer; +import org.apache.kafka.streams.kstream.internals.FilteredIterator; +import org.apache.kafka.streams.kstream.internals.WindowSupport; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streams.processor.internals.RecordCollector; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.RestoreFunc; +import org.apache.kafka.streams.processor.internals.Stamped; + +import java.util.Collections; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Map; + +public class SlidingWindowDef implements WindowDef { + private final String name; + private final long duration; + private final int maxCount; + private final Serializer keySerializer; + private final Serializer valueSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valueDeserializer; + + public SlidingWindowDef( + String name, + long duration, + int maxCount, + Serializer keySerializer, + Serializer valueSerializer, + Deserializer keyDeseriaizer, + Deserializer valueDeserializer) { + this.name = name; + this.duration = duration; + this.maxCount = maxCount; + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.keyDeserializer = keyDeseriaizer; + this.valueDeserializer = valueDeserializer; + } + + @Override + public String name() { + return name; + } + + @Override + public Window instance() { + return new SlidingWindow(); + } + + public class SlidingWindow extends WindowSupport implements Window { + private final Object lock = new Object(); + private ProcessorContext context; + private int slotNum; // used as a key for Kafka log compaction + private LinkedList list = new LinkedList(); + private HashMap> map = new HashMap<>(); + + @Override + public void init(ProcessorContext context) { + this.context = context; + RestoreFuncImpl restoreFunc = new RestoreFuncImpl(); + context.register(this, restoreFunc); + + for (ValueList valueList : map.values()) { + valueList.clearDirtyValues(); + } + this.slotNum = restoreFunc.slotNum; + } + + @Override + public Iterator findAfter(K key, final long timestamp) { + return find(key, timestamp, timestamp + duration); + } + + @Override + public Iterator findBefore(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp); + } + + @Override + public Iterator find(K key, final long timestamp) { + return find(key, timestamp - duration, timestamp + duration); + } + + /* + * finds items in the window between startTime and endTime (both inclusive) + */ + private Iterator find(K key, final long startTime, final long endTime) { + final ValueList values = map.get(key); + + if (values == null) { + return Collections.emptyIterator(); + } else { + return new FilteredIterator>(values.iterator()) { + @Override + protected V filter(Value item) { + if (startTime <= item.timestamp && item.timestamp <= endTime) + return item.value; + else + return null; + } + }; + } + } + + @Override + public void put(K key, V value, long timestamp) { + synchronized (lock) { + slotNum++; + + list.offerLast(key); + + ValueList values = map.get(key); + if (values == null) { + values = new ValueList<>(); + map.put(key, values); + } + + values.add(slotNum, value, timestamp); + } + evictExcess(); + evictExpired(timestamp - duration); + } + + private void evictExcess() { + while (list.size() > maxCount) { + K oldestKey = list.pollFirst(); + + ValueList values = map.get(oldestKey); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } + } + + private void evictExpired(long cutoffTime) { + while (true) { + K oldestKey = list.peekFirst(); + + ValueList values = map.get(oldestKey); + Stamped oldestValue = values.first(); + + if (oldestValue.timestamp < cutoffTime) { + list.pollFirst(); + values.removeFirst(); + + if (values.isEmpty()) map.remove(oldestKey); + } else { + break; + } + } + } + + @Override + public String name() { + return name; + } + + @Override + public void flush() { + IntegerSerializer intSerializer = new IntegerSerializer(); + ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); + + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + + for (Map.Entry> entry : map.entrySet()) { + ValueList values = entry.getValue(); + if (values.hasDirtyValues()) { + K key = entry.getKey(); + + byte[] keyBytes = keySerializer.serialize(name, key); + + Iterator> iterator = values.dirtyValueIterator(); + while (iterator.hasNext()) { + Value dirtyValue = iterator.next(); + byte[] slot = intSerializer.serialize("", dirtyValue.slotNum); + byte[] valBytes = valueSerializer.serialize(name, dirtyValue.value); + + byte[] combined = new byte[8 + 4 + keyBytes.length + 4 + valBytes.length]; + + int offset = 0; + offset += putLong(combined, offset, dirtyValue.timestamp); + offset += puts(combined, offset, keyBytes); + offset += puts(combined, offset, valBytes); + + if (offset != combined.length) + throw new IllegalStateException("serialized length does not match"); + + collector.send(new ProducerRecord<>(name, context.id(), slot, combined), byteArraySerializer, byteArraySerializer); + } + values.clearDirtyValues(); + } + } + } + + @Override + public void close() { + // TODO + } + + @Override + public boolean persistent() { + // TODO: should not be persistent, right? + return false; + } + + private class RestoreFuncImpl implements RestoreFunc { + + final IntegerDeserializer intDeserializer; + int slotNum = 0; + + RestoreFuncImpl() { + intDeserializer = new IntegerDeserializer(); + } + + @Override + public void apply(byte[] slot, byte[] bytes) { + + slotNum = intDeserializer.deserialize("", slot); + + int offset = 0; + // timestamp + long timestamp = getLong(bytes, offset); + offset += 8; + // key + int length = getInt(bytes, offset); + offset += 4; + K key = deserialize(bytes, offset, length, name, keyDeserializer); + offset += length; + // value + length = getInt(bytes, offset); + offset += 4; + V value = deserialize(bytes, offset, length, name, valueDeserializer); + + put(key, value, timestamp); + } + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java new file mode 100644 index 0000000000000..93fc359bcaa35 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java @@ -0,0 +1,23 @@ +/** + * 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.kstream; + +public interface ValueJoiner { + + R apply(V1 value1, V2 value2); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java new file mode 100644 index 0000000000000..a32423d04926c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java @@ -0,0 +1,23 @@ +/** + * 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.kstream; + +public interface ValueMapper { + + V2 apply(V1 value); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java new file mode 100644 index 0000000000000..a1456f6c421b6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -0,0 +1,36 @@ +/** + * 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.kstream; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; + +import java.util.Iterator; + +public interface Window extends StateStore { + + void init(ProcessorContext context); + + Iterator find(K key, long timestamp); + + Iterator findAfter(K key, long timestamp); + + Iterator findBefore(K key, long timestamp); + + void put(K key, V value, long timestamp); +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java new file mode 100644 index 0000000000000..bbc5979300b79 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/WindowDef.java @@ -0,0 +1,25 @@ +/** + * 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.kstream; + +public interface WindowDef { + + String name(); + + Window instance(); +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java new file mode 100644 index 0000000000000..54d44f01b8723 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/FilteredIterator.java @@ -0,0 +1,63 @@ +/** + * 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.kstream.internals; + +import java.util.Iterator; + +public abstract class FilteredIterator implements Iterator { + + private Iterator inner; + private T nextValue = null; + + public FilteredIterator(Iterator inner) { + this.inner = inner; + + findNext(); + } + + @Override + public boolean hasNext() { + return nextValue != null; + } + + @Override + public T next() { + T value = nextValue; + findNext(); + + return value; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + private void findNext() { + while (inner.hasNext()) { + S item = inner.next(); + nextValue = filter(item); + if (nextValue != null) { + return; + } + } + nextValue = null; + } + + protected abstract T filter(S item); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java new file mode 100644 index 0000000000000..6b661b423e60a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamBranch.java @@ -0,0 +1,52 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.kstream.Predicate; + +class KStreamBranch implements ProcessorDef { + + private final Predicate[] predicates; + + @SuppressWarnings("unchecked") + public KStreamBranch(Predicate... predicates) { + this.predicates = predicates; + } + + @Override + public Processor instance() { + return new KStreamBranchProcessor(); + } + + private class KStreamBranchProcessor extends AbstractProcessor { + @Override + public void process(K key, V value) { + for (int i = 0; i < predicates.length; i++) { + if (predicates[i].apply(key, value)) { + // use forward with childIndex here and then break the loop + // so that no record is going to be piped to multiple streams + context().forward(key, value, i); + break; + } + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java new file mode 100644 index 0000000000000..5444e70b8395e --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java @@ -0,0 +1,48 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.processor.ProcessorDef; + +class KStreamFilter implements ProcessorDef { + + private final Predicate predicate; + private final boolean filterOut; + + public KStreamFilter(Predicate predicate, boolean filterOut) { + this.predicate = predicate; + this.filterOut = filterOut; + } + + @Override + public Processor instance() { + return new KStreamFilterProcessor(); + } + + private class KStreamFilterProcessor extends AbstractProcessor { + @Override + public void process(K key, V value) { + if (filterOut ^ predicate.apply(key, value)) { + context().forward(key, value); + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java new file mode 100644 index 0000000000000..410cfda6366f7 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMap.java @@ -0,0 +1,47 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; + +class KStreamFlatMap implements ProcessorDef { + + private final KeyValueMapper>> mapper; + + KStreamFlatMap(KeyValueMapper>> mapper) { + this.mapper = mapper; + } + + @Override + public Processor instance() { + return new KStreamFlatMapProcessor(); + } + + private class KStreamFlatMapProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + for (KeyValue newPair : mapper.apply(key, value)) { + context().forward(newPair.key, newPair.value); + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java new file mode 100644 index 0000000000000..edca42142ed2b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValues.java @@ -0,0 +1,47 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; + +class KStreamFlatMapValues implements ProcessorDef { + + private final ValueMapper> mapper; + + KStreamFlatMapValues(ValueMapper> mapper) { + this.mapper = mapper; + } + + @Override + public Processor instance() { + return new KStreamFlatMapValuesProcessor(); + } + + private class KStreamFlatMapValuesProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + Iterable newValues = mapper.apply(value); + for (V2 v : newValues) { + context().forward(key, v); + } + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java new file mode 100644 index 0000000000000..69366481cd7b2 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -0,0 +1,201 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.kstream.KStreamWindowed; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.kstream.WindowDef; + +import java.lang.reflect.Array; +import java.util.concurrent.atomic.AtomicInteger; + +public class KStreamImpl implements KStream { + + private static final String FILTER_NAME = "KAFKA-FILTER-"; + + private static final String MAP_NAME = "KAFKA-MAP-"; + + private static final String MAPVALUES_NAME = "KAFKA-MAPVALUES-"; + + private static final String FLATMAP_NAME = "KAFKA-FLATMAP-"; + + private static final String FLATMAPVALUES_NAME = "KAFKA-FLATMAPVALUES-"; + + private static final String PROCESSOR_NAME = "KAFKA-PROCESSOR-"; + + private static final String BRANCH_NAME = "KAFKA-BRANCH-"; + + private static final String BRANCHCHILD_NAME = "KAFKA-BRANCHCHILD-"; + + private static final String WINDOWED_NAME = "KAFKA-WINDOWED-"; + + private static final String SINK_NAME = "KAFKA-SINK-"; + + public static final String JOINTHIS_NAME = "KAFKA-JOINTHIS-"; + + public static final String JOINOTHER_NAME = "KAFKA-JOINOTHER-"; + + public static final String JOINMERGE_NAME = "KAFKA-JOINMERGE-"; + + public static final String SOURCE_NAME = "KAFKA-SOURCE-"; + + public static final AtomicInteger INDEX = new AtomicInteger(1); + + protected final TopologyBuilder topology; + protected final String name; + + public KStreamImpl(TopologyBuilder topology, String name) { + this.topology = topology; + this.name = name; + } + + @Override + public KStream filter(Predicate predicate) { + String name = FILTER_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFilter<>(predicate, false), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream filterOut(final Predicate predicate) { + String name = FILTER_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFilter<>(predicate, true), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream map(KeyValueMapper> mapper) { + String name = MAP_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamMap<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream mapValues(ValueMapper mapper) { + String name = MAPVALUES_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamMapValues<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream flatMap(KeyValueMapper>> mapper) { + String name = FLATMAP_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFlatMap<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStream flatMapValues(ValueMapper> mapper) { + String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamFlatMapValues<>(mapper), this.name); + + return new KStreamImpl<>(topology, name); + } + + @Override + public KStreamWindowed with(WindowDef window) { + String name = WINDOWED_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, new KStreamWindow<>(window), this.name); + + return new KStreamWindowedImpl<>(topology, name, window); + } + + @Override + @SuppressWarnings("unchecked") + public KStream[] branch(Predicate... predicates) { + String branchName = BRANCH_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(branchName, new KStreamBranch(predicates.clone()), this.name); + + KStream[] branchChildren = (KStream[]) Array.newInstance(KStream.class, predicates.length); + for (int i = 0; i < predicates.length; i++) { + String childName = BRANCHCHILD_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(childName, new KStreamPassThrough(), branchName); + + branchChildren[i] = new KStreamImpl<>(topology, childName); + } + + return branchChildren; + } + + @Override + public KStream through(String topic, + Serializer keySerializer, + Serializer valSerializer, + Deserializer keyDeserializer, + Deserializer valDeserializer) { + String sendName = SINK_NAME + INDEX.getAndIncrement(); + + topology.addSink(sendName, topic, keySerializer, valSerializer, this.name); + + String sourceName = SOURCE_NAME + INDEX.getAndIncrement(); + + topology.addSource(sourceName, keyDeserializer, valDeserializer, topic); + + return new KStreamImpl<>(topology, sourceName); + } + + @Override + public KStream through(String topic) { + return through(topic, (Serializer) null, (Serializer) null, (Deserializer) null, (Deserializer) null); + } + + @Override + public void to(String topic) { + String name = SINK_NAME + INDEX.getAndIncrement(); + + topology.addSink(name, topic, this.name); + } + + @Override + public void to(String topic, Serializer keySerializer, Serializer valSerializer) { + String name = SINK_NAME + INDEX.getAndIncrement(); + + topology.addSink(name, topic, keySerializer, valSerializer, this.name); + } + + @Override + public KStream process(final ProcessorDef processorDef) { + String name = PROCESSOR_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(name, processorDef, this.name); + + return new KStreamImpl<>(topology, name); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java new file mode 100644 index 0000000000000..4003d29f8a874 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoin.java @@ -0,0 +1,96 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorDef; + +import java.util.Iterator; + +class KStreamJoin implements ProcessorDef { + + private static abstract class Finder { + abstract Iterator find(K key, long timestamp); + } + + private final String windowName; + private final ValueJoiner joiner; + + KStreamJoin(String windowName, ValueJoiner joiner) { + this.windowName = windowName; + this.joiner = joiner; + } + + @Override + public Processor instance() { + return new KStreamJoinProcessor(windowName); + } + + private class KStreamJoinProcessor extends AbstractProcessor { + + private final String windowName; + protected Finder finder; + + public KStreamJoinProcessor(String windowName) { + this.windowName = windowName; + } + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + super.init(context); + + // check if these two streams are joinable + if (!context.joinable()) + throw new IllegalStateException("Streams are not joinable."); + + final Window window = (Window) context.getStateStore(windowName); + + this.finder = new Finder() { + Iterator find(K key, long timestamp) { + return window.find(key, timestamp); + } + }; + } + + @Override + public void process(K key, V1 value) { + long timestamp = context().timestamp(); + Iterator iter = finder.find(key, timestamp); + if (iter != null) { + while (iter.hasNext()) { + context().forward(key, joiner.apply(value, iter.next())); + } + } + } + } + + public static ValueJoiner reverseJoiner(final ValueJoiner joiner) { + return new ValueJoiner() { + @Override + public R apply(T2 value2, T1 value1) { + return joiner.apply(value1, value2); + } + }; + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java new file mode 100644 index 0000000000000..4d3134874f3e2 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMap.java @@ -0,0 +1,46 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.processor.ProcessorDef; + +class KStreamMap implements ProcessorDef { + + private final KeyValueMapper> mapper; + + public KStreamMap(KeyValueMapper> mapper) { + this.mapper = mapper; + } + + @Override + public Processor instance() { + return new KStreamMapProcessor(); + } + + private class KStreamMapProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + KeyValue newPair = mapper.apply(key, value); + context().forward(newPair.key, newPair.value); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java new file mode 100644 index 0000000000000..dac655096fcc6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamMapValues.java @@ -0,0 +1,45 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.ProcessorDef; + +class KStreamMapValues implements ProcessorDef { + + private final ValueMapper mapper; + + public KStreamMapValues(ValueMapper mapper) { + this.mapper = mapper; + } + + @Override + public Processor instance() { + return new KStreamMapProcessor(); + } + + private class KStreamMapProcessor extends AbstractProcessor { + @Override + public void process(K1 key, V1 value) { + V2 newValue = mapper.apply(value); + context().forward(key, newValue); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java new file mode 100644 index 0000000000000..ea395503534e4 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamPassThrough.java @@ -0,0 +1,37 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; + +class KStreamPassThrough implements ProcessorDef { + + @Override + public Processor instance() { + return new KStreamPassThroughProcessor(); + } + + public class KStreamPassThroughProcessor extends AbstractProcessor { + @Override + public void process(K key, V value) { + context().forward(key, value); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java new file mode 100644 index 0000000000000..6ebcbe16e4d1b --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindow.java @@ -0,0 +1,68 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.kstream.WindowDef; + +public class KStreamWindow implements ProcessorDef { + + private final WindowDef windowDef; + + KStreamWindow(WindowDef windowDef) { + this.windowDef = windowDef; + } + + public WindowDef window() { + return windowDef; + } + + @Override + public Processor instance() { + return new KStreamWindowProcessor(); + } + + private class KStreamWindowProcessor extends AbstractProcessor { + + private Window window; + + @Override + public void init(ProcessorContext context) { + super.init(context); + this.window = windowDef.instance(); + this.window.init(context); + } + + @Override + public void process(K key, V value) { + synchronized (this) { + window.put(key, value, context().timestamp()); + context().forward(key, value); + } + } + + @Override + public void close() { + window.close(); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java new file mode 100644 index 0000000000000..a208af6cca137 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedImpl.java @@ -0,0 +1,54 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamWindowed; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.streams.processor.TopologyBuilder; + +public final class KStreamWindowedImpl extends KStreamImpl implements KStreamWindowed { + + private final WindowDef windowDef; + + public KStreamWindowedImpl(TopologyBuilder topology, String name, WindowDef windowDef) { + super(topology, name); + this.windowDef = windowDef; + } + + @Override + public KStream join(KStreamWindowed other, ValueJoiner valueJoiner) { + String thisWindowName = this.windowDef.name(); + String otherWindowName = ((KStreamWindowedImpl) other).windowDef.name(); + + KStreamJoin joinThis = new KStreamJoin<>(otherWindowName, valueJoiner); + KStreamJoin joinOther = new KStreamJoin<>(thisWindowName, KStreamJoin.reverseJoiner(valueJoiner)); + KStreamPassThrough joinMerge = new KStreamPassThrough<>(); + + String joinThisName = JOINTHIS_NAME + INDEX.getAndIncrement(); + String joinOtherName = JOINOTHER_NAME + INDEX.getAndIncrement(); + String joinMergeName = JOINMERGE_NAME + INDEX.getAndIncrement(); + + topology.addProcessor(joinThisName, joinThis, this.name); + topology.addProcessor(joinOtherName, joinOther, ((KStreamImpl) other).name); + topology.addProcessor(joinMergeName, joinMerge, joinThisName, joinOtherName); + + return new KStreamImpl<>(topology, joinMergeName); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java new file mode 100644 index 0000000000000..b54bcc998c539 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowSupport.java @@ -0,0 +1,159 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.processor.internals.Stamped; + +import java.util.Iterator; + +public class WindowSupport { + + public static class ValueList { + Value head = null; + Value tail = null; + Value dirty = null; + + public void add(int slotNum, V value, long timestamp) { + Value v = new Value<>(slotNum, value, timestamp); + if (tail != null) { + tail.next = v; + } else { + head = v; + } + tail = v; + if (dirty == null) dirty = v; + } + + public Value first() { + return head; + } + + public void removeFirst() { + if (head != null) { + if (head == tail) tail = null; + head = head.next; + } + } + + public boolean isEmpty() { + return head == null; + } + + public boolean hasDirtyValues() { + return dirty != null; + } + + public void clearDirtyValues() { + dirty = null; + } + + public Iterator> iterator() { + return new ValueListIterator(head); + } + + public Iterator> dirtyValueIterator() { + return new ValueListIterator(dirty); + } + + } + + private static class ValueListIterator implements Iterator> { + + Value ptr; + + ValueListIterator(Value start) { + ptr = start; + } + + @Override + public boolean hasNext() { + return ptr != null; + } + + @Override + public Value next() { + Value value = ptr; + if (value != null) ptr = value.next; + return value; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + + } + + public static class Value extends Stamped { + public final int slotNum; + private Value next = null; + + Value(int slotNum, V value, long timestamp) { + super(value, timestamp); + this.slotNum = slotNum; + } + } + + + public static long getLong(byte[] bytes, int offset) { + long value = 0; + for (int i = 0; i < 8; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int getInt(byte[] bytes, int offset) { + int value = 0; + for (int i = 0; i < 4; i++) { + value = (value << 8) | bytes[offset + i]; + } + return value; + } + + public static int putLong(byte[] bytes, int offset, long value) { + for (int i = 7; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 8; + } + + public static int putInt(byte[] bytes, int offset, int value) { + for (int i = 3; i >= 0; i--) { + bytes[offset + i] = (byte) (value & 0xFF); + value = value >> 8; + } + return 4; + } + + public static int puts(byte[] bytes, int offset, byte[] value) { + offset += putInt(bytes, offset, value.length); + System.arraycopy(bytes, offset, value, 0, value.length); + return 4 + value.length; + } + + + public static T deserialize(byte[] bytes, int offset, int length, String topic, Deserializer deserializer) { + byte[] buf = new byte[length]; + System.arraycopy(bytes, offset, buf, 0, length); + return deserializer.deserialize(topic, buf); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java new file mode 100644 index 0000000000000..01d0024d64d78 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/AbstractProcessor.java @@ -0,0 +1,71 @@ +/** + * 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.processor; + +/** + * An abstract implementation of {@link Processor} that manages the {@link ProcessorContext} instance and provides default no-op + * implementations of {@link #punctuate(long)} and {@link #close()}. + * + * @param the type of keys + * @param the type of values + */ +public abstract class AbstractProcessor implements Processor { + + private ProcessorContext context; + + protected AbstractProcessor() { + } + + @Override + public void init(ProcessorContext context) { + this.context = context; + } + + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + *

+ * This method does nothing by default; if desired, subclasses should override it with custom functionality. + *

+ * + * @param streamTime the stream time when this method is being called + */ + @Override + public void punctuate(long streamTime) { + // do nothing + } + + /** + * Close this processor and clean up any resources. + *

+ * This method does nothing by default; if desired, subclasses should override it with custom functionality. + *

+ */ + @Override + public void close() { + // do nothing + } + + /** + * Get the processor's context set during {@link #init(ProcessorContext) initialization}. + * + * @return the processor context; null only when called prior to {@link #init(ProcessorContext) initialization}. + */ + protected final ProcessorContext context() { + return this.context; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java new file mode 100644 index 0000000000000..3cade3a907424 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -0,0 +1,59 @@ +/** + * 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.processor; + +/** + * A processor of messages. + * + * @param the type of keys + * @param the type of values + */ +public interface Processor { + + /** + * Initialize this processor with the given context. The framework ensures this is called once per processor when the topology + * that contains it is initialized. + *

+ * If this processor is to be {@link #punctuate(long) called periodically} by the framework, then this method should + * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. + * + * @param context the context; may not be null + */ + void init(ProcessorContext context); + + /** + * Process the message with the given key and value. + * + * @param key the key for the message + * @param value the value for the message + */ + void process(K key, V value); + + /** + * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * during {@link #init(ProcessorContext) initialization}. + * + * @param timestamp the stream time when this method is being called + */ + void punctuate(long timestamp); + + /** + * Close this processor and clean up any resources. + */ + void close(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java new file mode 100644 index 0000000000000..6b32b83abbff0 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -0,0 +1,106 @@ +/** + * 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.processor; + +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.io.File; + +public interface ProcessorContext { + + /** + * Returns the partition group id + * + * @return partition group id + */ + int id(); + + /** + * Returns the key serializer + * + * @return the key serializer + */ + Serializer keySerializer(); + + /** + * Returns the value serializer + * + * @return the value serializer + */ + Serializer valueSerializer(); + + /** + * Returns the key deserializer + * + * @return the key deserializer + */ + Deserializer keyDeserializer(); + + /** + * Returns the value deserializer + * + * @return the value deserializer + */ + Deserializer valueDeserializer(); + + /** + * Returns the state directory for the partition. + * + * @return the state directory + */ + File stateDir(); + + /** + * Returns Metrics instance + * + * @return Metrics + */ + Metrics metrics(); + + /** + * Check if this process's incoming streams are joinable + */ + boolean joinable(); + + /** + * Registers and possibly restores the specified storage engine. + * + * @param store the storage engine + */ + void register(StateStore store, RestoreFunc restoreFunc); + + StateStore getStateStore(String name); + + void schedule(long interval); + + void forward(K key, V value); + + void forward(K key, V value, int childIndex); + + void commit(); + + String topic(); + + int partition(); + + long offset(); + + long timestamp(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorDef.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorDef.java new file mode 100644 index 0000000000000..99f029989127c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorDef.java @@ -0,0 +1,23 @@ +/** + * 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.processor; + +public interface ProcessorDef { + + Processor instance(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/RestoreFunc.java b/streams/src/main/java/org/apache/kafka/streams/processor/RestoreFunc.java new file mode 100644 index 0000000000000..883147e2b5386 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/RestoreFunc.java @@ -0,0 +1,27 @@ +/** + * 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.processor; + +/** + * Restoration logic for log-backed state stores upon restart, + * it takes one record at a time from the logs to apply to the restoring state. + */ +public interface RestoreFunc { + + void apply(byte[] key, byte[] value); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java new file mode 100644 index 0000000000000..38afe9bacad67 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java @@ -0,0 +1,52 @@ +/** + * 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.processor; + +/** + * A storage engine for managing state maintained by a stream processor. + * + *

+ * This interface does not specify any query capabilities, which, of course, + * would be query engine specific. Instead it just specifies the minimum + * functionality required to reload a storage engine from its changelog as well + * as basic lifecycle management. + *

+ */ +public interface StateStore { + + /** + * The name of this store. + * @return the storage name + */ + String name(); + + /** + * Flush any cached data + */ + void flush(); + + /** + * Close the storage engine + */ + void close(); + + /** + * If the storage is persistent + */ + boolean persistent(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java new file mode 100644 index 0000000000000..62098f21d56df --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java @@ -0,0 +1,34 @@ +/** + * 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.processor; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +/** + * An interface that allows the KStream framework to extract a timestamp from a key-value pair + */ +public interface TimestampExtractor { + + /** + * Extracts a timestamp from a message + * + * @param record ConsumerRecord + * @return timestamp + */ + long extract(ConsumerRecord record); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java new file mode 100644 index 0000000000000..a254c131c1a47 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -0,0 +1,293 @@ +/** + * 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.processor; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.internals.ProcessorNode; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.SinkNode; +import org.apache.kafka.streams.processor.internals.SourceNode; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A component that is used to build a {@link ProcessorTopology}. A topology contains an acyclic graph of sources, processors, + * and sinks. A {@link SourceNode source} is a node in the graph that consumes one or more Kafka topics and forwards them to + * its child nodes. A {@link Processor processor} is a node in the graph that receives input messages from upstream nodes, + * processes that message, and optionally forwarding new messages to one or all of its children. Finally, a {@link SinkNode sink} + * is a node in the graph that receives messages from upstream nodes and writes them to a Kafka topic. This builder allows you + * to construct an acyclic graph of these nodes, and the builder is then passed into a new {@link KafkaStreaming} instance + * that will then {@link KafkaStreaming#start() begin consuming, processing, and producing messages}. + */ +public class TopologyBuilder { + + // list of node factories in a topological order + private ArrayList nodeFactories = new ArrayList<>(); + + private Set nodeNames = new HashSet<>(); + private Set sourceTopicNames = new HashSet<>(); + + private interface NodeFactory { + ProcessorNode build(); + } + + private class ProcessorNodeFactory implements NodeFactory { + public final String[] parents; + private final String name; + private final ProcessorDef definition; + + public ProcessorNodeFactory(String name, String[] parents, ProcessorDef definition) { + this.name = name; + this.parents = parents.clone(); + this.definition = definition; + } + + @Override + public ProcessorNode build() { + Processor processor = definition.instance(); + return new ProcessorNode(name, processor); + } + } + + private class SourceNodeFactory implements NodeFactory { + public final String[] topics; + private final String name; + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + + private SourceNodeFactory(String name, String[] topics, Deserializer keyDeserializer, Deserializer valDeserializer) { + this.name = name; + this.topics = topics.clone(); + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + + @Override + public ProcessorNode build() { + return new SourceNode(name, keyDeserializer, valDeserializer); + } + } + + private class SinkNodeFactory implements NodeFactory { + public final String[] parents; + public final String topic; + private final String name; + private Serializer keySerializer; + private Serializer valSerializer; + + private SinkNodeFactory(String name, String[] parents, String topic, Serializer keySerializer, Serializer valSerializer) { + this.name = name; + this.parents = parents.clone(); + this.topic = topic; + this.keySerializer = keySerializer; + this.valSerializer = valSerializer; + } + @Override + public ProcessorNode build() { + return new SinkNode(name, topic, keySerializer, valSerializer); + } + } + + /** + * Create a new builder. + */ + public TopologyBuilder() {} + + /** + * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. + * The source will use the {@link StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} and + * {@link StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * {@link StreamingConfig streaming configuration}. + * + * @param name the unique name of the source used to reference this node when + * {@link #addProcessor(String, ProcessorDef, String...) adding processor children}. + * @param topics the name of one or more Kafka topics that this source is to consume + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSource(String name, String... topics) { + return addSource(name, (Deserializer) null, (Deserializer) null, topics); + } + + /** + * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. + * The sink will use the specified key and value deserializers. + * + * @param name the unique name of the source used to reference this node when + * {@link #addProcessor(String, ProcessorDef, String...) adding processor children}. + * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming messages; may be null if the source + * should use the {@link StreamingConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param valDeserializer the {@link Deserializer value deserializer} used when consuming messages; may be null if the source + * should use the {@link StreamingConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param topics the name of one or more Kafka topics that this source is to consume + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + if (nodeNames.contains(name)) + throw new TopologyException("Processor " + name + " is already added."); + + for (String topic : topics) { + if (sourceTopicNames.contains(topic)) + throw new TopologyException("Topic " + topic + " has already been registered by another source."); + + sourceTopicNames.add(topic); + } + + nodeNames.add(name); + nodeFactories.add(new SourceNodeFactory(name, topics, keyDeserializer, valDeserializer)); + return this; + } + + /** + * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. + * The sink will use the {@link StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and + * {@link StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * {@link StreamingConfig streaming configuration}. + * + * @param name the unique name of the sink + * @param topic the name of the Kafka topic to which this sink should write its messages + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSink(String name, String topic, String... parentNames) { + return addSink(name, topic, (Serializer) null, (Serializer) null, parentNames); + } + + /** + * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. + * The sink will use the specified key and value serializers. + * + * @param name the unique name of the sink + * @param topic the name of the Kafka topic to which this sink should write its messages + * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink + * should use the {@link StreamingConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink + * should use the {@link StreamingConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * {@link StreamingConfig streaming configuration} + * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume + * and write to its topic + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, String... parentNames) { + if (nodeNames.contains(name)) + throw new TopologyException("Processor " + name + " is already added."); + + if (parentNames != null) { + for (String parent : parentNames) { + if (parent.equals(name)) { + throw new TopologyException("Processor " + name + " cannot be a parent of itself."); + } + if (!nodeNames.contains(parent)) { + throw new TopologyException("Parent processor " + parent + " is not added yet."); + } + } + } + + nodeNames.add(name); + nodeFactories.add(new SinkNodeFactory(name, parentNames, topic, keySerializer, valSerializer)); + return this; + } + + /** + * Add a new processor node that receives and processes messages output by one or more parent source or processor node. + * Any new messages output by this processor will be forwarded to its child processor or sink nodes. + * @param name the unique name of the processor node + * @param definition the supplier used to obtain this node's {@link Processor} instance + * @param parentNames the name of one or more source or processor nodes whose output messages this processor should receive + * and process + * @return this builder instance so methods can be chained together; never null + */ + public final TopologyBuilder addProcessor(String name, ProcessorDef definition, String... parentNames) { + if (nodeNames.contains(name)) + throw new TopologyException("Processor " + name + " is already added."); + + if (parentNames != null) { + for (String parent : parentNames) { + if (parent.equals(name)) { + throw new TopologyException("Processor " + name + " cannot be a parent of itself."); + } + if (!nodeNames.contains(parent)) { + throw new TopologyException("Parent processor " + parent + " is not added yet."); + } + } + } + + nodeNames.add(name); + nodeFactories.add(new ProcessorNodeFactory(name, parentNames, definition)); + return this; + } + + /** + * Build the topology. This is typically called automatically when passing this builder into the + * {@link KafkaStreaming#KafkaStreaming(TopologyBuilder, StreamingConfig)} constructor. + * + * @see KafkaStreaming#KafkaStreaming(TopologyBuilder, StreamingConfig) + */ + @SuppressWarnings("unchecked") + public ProcessorTopology build() { + List processorNodes = new ArrayList<>(nodeFactories.size()); + Map processorMap = new HashMap<>(); + Map topicSourceMap = new HashMap<>(); + + try { + // create processor nodes in a topological order ("nodeFactories" is already topologically sorted) + for (NodeFactory factory : nodeFactories) { + ProcessorNode node = factory.build(); + processorNodes.add(node); + processorMap.put(node.name(), node); + + if (factory instanceof ProcessorNodeFactory) { + for (String parent : ((ProcessorNodeFactory) factory).parents) { + processorMap.get(parent).addChild(node); + } + } else if (factory instanceof SourceNodeFactory) { + for (String topic : ((SourceNodeFactory) factory).topics) { + topicSourceMap.put(topic, (SourceNode) node); + } + } else if (factory instanceof SinkNodeFactory) { + for (String parent : ((SinkNodeFactory) factory).parents) { + processorMap.get(parent).addChild(node); + } + } else { + throw new TopologyException("Unknown definition class: " + factory.getClass().getName()); + } + } + } catch (Exception e) { + throw new KafkaException("ProcessorNode construction failed: this should not happen."); + } + + return new ProcessorTopology(processorNodes, topicSourceMap); + } + + /** + * Get the names of topics that are to be consumed by the source nodes created by this builder. + * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null + */ + public Set sourceTopics() { + return Collections.unmodifiableSet(sourceTopicNames); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java new file mode 100644 index 0000000000000..99d14058a1f7a --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyException.java @@ -0,0 +1,38 @@ +/** + * 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.processor; + +import org.apache.kafka.common.KafkaException; + +public class TopologyException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public TopologyException(String message) { + super(message); + } + + public TopologyException(String name, Object value) { + this(name, value, null); + } + + public TopologyException(String name, Object value, String message) { + super("Invalid topology building" + (message == null ? "" : ": " + message)); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java new file mode 100644 index 0000000000000..717df2c54abcd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java @@ -0,0 +1,67 @@ +/** + * 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.processor.internals; + +import java.util.LinkedList; + +/** + * MinTimestampTracker implements {@link TimestampTracker} that maintains the min + * timestamp of the maintained stamped elements. + */ +public class MinTimestampTracker implements TimestampTracker { + + private final LinkedList> descendingSubsequence = new LinkedList<>(); + + // in the case that incoming traffic is very small, the records maybe put and polled + // within a single iteration, in this case we need to remember the last polled + // record's timestamp + private long lastKnownTime = NOT_KNOWN; + + public void addElement(Stamped elem) { + if (elem == null) throw new NullPointerException(); + + Stamped minElem = descendingSubsequence.peekLast(); + while (minElem != null && minElem.timestamp >= elem.timestamp) { + descendingSubsequence.removeLast(); + minElem = descendingSubsequence.peekLast(); + } + descendingSubsequence.offerLast(elem); + } + + public void removeElement(Stamped elem) { + if (elem != null && descendingSubsequence.peekFirst() == elem) + descendingSubsequence.removeFirst(); + + if (descendingSubsequence.isEmpty()) + lastKnownTime = elem.timestamp; + } + + public int size() { + return descendingSubsequence.size(); + } + + public long get() { + Stamped stamped = descendingSubsequence.peekFirst(); + + if (stamped == null) + return lastKnownTime; + else + return stamped.timestamp; + } + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java new file mode 100644 index 0000000000000..44a6c5c1a2f3d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -0,0 +1,165 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.processor.TimestampExtractor; + +import java.util.Collections; +import java.util.Comparator; +import java.util.Map; +import java.util.PriorityQueue; +import java.util.Set; + +/** + * A PartitionGroup is composed from a set of partitions. It also maintains the timestamp of this + * group, hence the associated task as the min timestamp across all partitions in the group. + */ +public class PartitionGroup { + + private final Map partitionQueues; + + private final PriorityQueue queuesByTime; + + private final TimestampExtractor timestampExtractor; + + public static class RecordInfo { + public RecordQueue queue; + + public ProcessorNode node() { + return queue.source(); + } + + public TopicPartition partition() { + return queue.partition(); + } + } + + // since task is thread-safe, we do not need to synchronize on local variables + private int totalBuffered; + + public PartitionGroup(Map partitionQueues, TimestampExtractor timestampExtractor) { + this.queuesByTime = new PriorityQueue<>(partitionQueues.size(), new Comparator() { + + @Override + public int compare(RecordQueue queue1, RecordQueue queue2) { + long time1 = queue1.timestamp(); + long time2 = queue2.timestamp(); + + if (time1 < time2) return -1; + if (time1 > time2) return 1; + return 0; + } + }); + + this.partitionQueues = partitionQueues; + + this.timestampExtractor = timestampExtractor; + + this.totalBuffered = 0; + } + + /** + * Get the next record and queue + * + * @return StampedRecord + */ + public StampedRecord nextRecord(RecordInfo info) { + StampedRecord record = null; + + RecordQueue queue = queuesByTime.poll(); + if (queue != null) { + // get the first record from this queue. + record = queue.poll(); + + if (queue.size() > 0) { + queuesByTime.offer(queue); + } + } + info.queue = queue; + + if (record != null) totalBuffered--; + + return record; + } + + /** + * Adds raw records to this partition group + * + * @param partition the partition + * @param rawRecords the raw records + * @return the queue size for the partition + */ + public int addRawRecords(TopicPartition partition, Iterable> rawRecords) { + RecordQueue recordQueue = partitionQueues.get(partition); + + int oldSize = recordQueue.size(); + int newSize = recordQueue.addRawRecords(rawRecords, timestampExtractor); + + // add this record queue to be considered for processing in the future if it was empty before + if (oldSize == 0 && newSize > 0) { + queuesByTime.offer(recordQueue); + } + + totalBuffered += newSize - oldSize; + + return newSize; + } + + public Set partitions() { + return Collections.unmodifiableSet(partitionQueues.keySet()); + } + + /** + * Return the timestamp of this partition group as the smallest + * partition timestamp among all its partitions + */ + public long timestamp() { + if (queuesByTime.isEmpty()) { + // if there is no data in all partitions, return the smallest of their last known times + long timestamp = Long.MAX_VALUE; + for (RecordQueue queue : partitionQueues.values()) { + if (timestamp > queue.timestamp()) + timestamp = queue.timestamp(); + } + return timestamp; + } else { + return queuesByTime.peek().timestamp(); + } + } + + public int numBuffered(TopicPartition partition) { + RecordQueue recordQueue = partitionQueues.get(partition); + + if (recordQueue == null) + throw new KafkaException("Record's partition does not belong to this partition-group."); + + return recordQueue.size(); + } + + public int numBuffered() { + return totalBuffered; + } + + public void close() { + queuesByTime.clear(); + partitionQueues.clear(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java new file mode 100644 index 0000000000000..b3502220cf3ed --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -0,0 +1,214 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.RestoreFunc; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ProcessorContextImpl implements ProcessorContext { + + private static final Logger log = LoggerFactory.getLogger(ProcessorContextImpl.class); + + private final int id; + private final StreamTask task; + private final Metrics metrics; + private final RecordCollector collector; + private final ProcessorStateManager stateMgr; + + private final Serializer keySerializer; + private final Serializer valSerializer; + private final Deserializer keyDeserializer; + private final Deserializer valDeserializer; + + private boolean initialized; + + @SuppressWarnings("unchecked") + public ProcessorContextImpl(int id, + StreamTask task, + StreamingConfig config, + RecordCollector collector, + ProcessorStateManager stateMgr, + Metrics metrics) { + this.id = id; + this.task = task; + this.metrics = metrics; + this.collector = collector; + this.stateMgr = stateMgr; + + this.keySerializer = config.getConfiguredInstance(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.valSerializer = config.getConfiguredInstance(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); + this.keyDeserializer = config.getConfiguredInstance(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + this.valDeserializer = config.getConfiguredInstance(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + + this.initialized = false; + } + + public RecordCollector recordCollector() { + return this.collector; + } + + public void initialized() { + this.initialized = true; + } + + @Override + public boolean joinable() { + Set partitions = this.task.partitions(); + Map> partitionsById = new HashMap<>(); + int firstId = -1; + for (TopicPartition partition : partitions) { + if (!partitionsById.containsKey(partition.partition())) { + partitionsById.put(partition.partition(), new ArrayList()); + } + partitionsById.get(partition.partition()).add(partition.topic()); + + if (firstId < 0) + firstId = partition.partition(); + } + + List topics = partitionsById.get(firstId); + for (List topicsPerPartition : partitionsById.values()) { + if (topics.size() != topicsPerPartition.size()) + return false; + + for (String topic : topicsPerPartition) { + if (!topics.contains(topic)) + return false; + } + } + + return true; + } + + @Override + public int id() { + return id; + } + + @Override + public Serializer keySerializer() { + return this.keySerializer; + } + + @Override + public Serializer valueSerializer() { + return this.valSerializer; + } + + @Override + public Deserializer keyDeserializer() { + return this.keyDeserializer; + } + + @Override + public Deserializer valueDeserializer() { + return this.valDeserializer; + } + + @Override + public File stateDir() { + return stateMgr.baseDir(); + } + + @Override + public Metrics metrics() { + return metrics; + } + + @Override + public void register(StateStore store, RestoreFunc restoreFunc) { + if (initialized) + throw new KafkaException("Can only create state stores during initialization."); + + stateMgr.register(store, restoreFunc); + } + + @Override + public StateStore getStateStore(String name) { + return stateMgr.getStore(name); + } + + @Override + public String topic() { + if (task.record() == null) + throw new IllegalStateException("this should not happen as topic() should only be called while a record is processed"); + + return task.record().topic(); + } + + @Override + public int partition() { + if (task.record() == null) + throw new IllegalStateException("this should not happen as partition() should only be called while a record is processed"); + + return task.record().partition(); + } + + @Override + public long offset() { + if (this.task.record() == null) + throw new IllegalStateException("this should not happen as offset() should only be called while a record is processed"); + + return this.task.record().offset(); + } + + @Override + public long timestamp() { + if (task.record() == null) + throw new IllegalStateException("this should not happen as timestamp() should only be called while a record is processed"); + + return task.record().timestamp; + } + + @Override + public void forward(K key, V value) { + task.forward(key, value); + } + + @Override + public void forward(K key, V value, int childIndex) { + task.forward(key, value, childIndex); + } + + @Override + public void commit() { + task.needCommit(); + } + + @Override + public void schedule(long interval) { + task.schedule(interval); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java new file mode 100644 index 0000000000000..9127c3f170e63 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -0,0 +1,70 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; + +import java.util.ArrayList; +import java.util.List; + +public class ProcessorNode { + + private final List> children; + + private final String name; + private final Processor processor; + + public ProcessorNode(String name) { + this(name, null); + } + + public ProcessorNode(String name, Processor processor) { + this.name = name; + this.processor = processor; + this.children = new ArrayList<>(); + } + + public final String name() { + return name; + } + + public final Processor processor() { + return processor; + } + + public final List> children() { + return children; + } + + public void addChild(ProcessorNode child) { + children.add(child); + } + + public void init(ProcessorContext context) { + processor.init(context); + } + + public void process(K key, V value) { + processor.process(key, value); + } + + public void close() { + processor.close(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java new file mode 100644 index 0000000000000..2f1fb35894170 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -0,0 +1,232 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.RestoreFunc; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.state.OffsetCheckpoint; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.RandomAccessFile; +import java.nio.channels.FileChannel; +import java.nio.channels.FileLock; +import java.nio.channels.OverlappingFileLockException; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +public class ProcessorStateManager { + + private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); + + public static final String CHECKPOINT_FILE_NAME = ".checkpoint"; + public static final String LOCK_FILE_NAME = ".lock"; + + private final int id; + private final File baseDir; + private final FileLock directoryLock; + private final Map stores; + private final Consumer restoreConsumer; + private final Map restoredOffsets; + private final Map checkpointedOffsets; + + public ProcessorStateManager(int id, File baseDir, Consumer restoreConsumer) throws IOException { + this.id = id; + this.baseDir = baseDir; + this.stores = new HashMap<>(); + this.restoreConsumer = restoreConsumer; + this.restoredOffsets = new HashMap<>(); + + // create the state directory for this task if missing (we won't create the parent directory) + createStateDirectory(baseDir); + + // try to acquire the exclusive lock on the state directory + directoryLock = lockStateDirectory(baseDir); + if (directoryLock == null) { + throw new IOException("Failed to lock the state directory: " + baseDir.getCanonicalPath()); + } + + // load the checkpoint information + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + this.checkpointedOffsets = new HashMap<>(checkpoint.read()); + + // delete the checkpoint file after finish loading its stored offsets + checkpoint.delete(); + } + + private static void createStateDirectory(File stateDir) throws IOException { + if (!stateDir.exists()) { + stateDir.mkdir(); + } + } + + public static FileLock lockStateDirectory(File stateDir) throws IOException { + File lockFile = new File(stateDir, ProcessorStateManager.LOCK_FILE_NAME); + FileChannel channel = new RandomAccessFile(lockFile, "rw").getChannel(); + try { + return channel.tryLock(); + } catch (OverlappingFileLockException e) { + return null; + } + } + + public File baseDir() { + return this.baseDir; + } + + public void register(StateStore store, RestoreFunc restoreFunc) { + if (store.name().equals(CHECKPOINT_FILE_NAME)) + throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); + + if (this.stores.containsKey(store.name())) + throw new IllegalArgumentException("Store " + store.name() + " has already been registered."); + + // ---- register the store ---- // + + // check that the underlying change log topic exist or not + if (restoreConsumer.listTopics().containsKey(store.name())) { + boolean partitionNotFound = true; + for (PartitionInfo partitionInfo : restoreConsumer.partitionsFor(store.name())) { + if (partitionInfo.partition() == id) { + partitionNotFound = false; + break; + } + } + + if (partitionNotFound) + throw new IllegalStateException("Store " + store.name() + "'s change log does not contain the partition for group " + id); + + } else { + throw new IllegalStateException("Change log topic for store " + store.name() + " does not exist yet"); + } + + this.stores.put(store.name(), store); + + // ---- try to restore the state from change-log ---- // + + // subscribe to the store's partition + TopicPartition storePartition = new TopicPartition(store.name(), id); + if (!restoreConsumer.subscription().isEmpty()) { + throw new IllegalStateException("Restore consumer should have not subscribed to any partitions beforehand"); + } + restoreConsumer.assign(Collections.singletonList(storePartition)); + + // calculate the end offset of the partition + // TODO: this is a bit hacky to first seek then position to get the end offset + restoreConsumer.seekToEnd(storePartition); + long endOffset = restoreConsumer.position(storePartition); + + // load the previously flushed state and restore from the checkpointed offset of the change log + // if it exists in the offset file; restore the state from the beginning of the change log otherwise + if (checkpointedOffsets.containsKey(storePartition)) { + restoreConsumer.seek(storePartition, checkpointedOffsets.get(storePartition)); + } else { + // TODO: in this case, we need to ignore the preciously flushed state + restoreConsumer.seekToBeginning(storePartition); + } + + // restore its state from changelog records; while restoring the log end offset + // should not change since it is only written by this thread. + while (true) { + for (ConsumerRecord record : restoreConsumer.poll(100).records(storePartition)) { + restoreFunc.apply(record.key(), record.value()); + } + + if (restoreConsumer.position(storePartition) == endOffset) { + break; + } else if (restoreConsumer.position(storePartition) > endOffset) { + throw new IllegalStateException("Log end offset should not change while restoring"); + } + } + + // record the restored offset for its change log partition + long newOffset = restoreConsumer.position(storePartition); + restoredOffsets.put(storePartition, newOffset); + + // un-assign the change log partition + restoreConsumer.assign(Collections.emptyList()); + } + + public StateStore getStore(String name) { + return stores.get(name); + } + + public void cleanup() throws IOException { + // clean up any unknown files in the state directory + for (File file : this.baseDir.listFiles()) { + if (!this.stores.containsKey(file.getName())) { + log.info("Deleting state directory {}", file.getAbsolutePath()); + file.delete(); + } + } + } + + public void flush() { + if (!this.stores.isEmpty()) { + log.debug("Flushing stores."); + for (StateStore store : this.stores.values()) + store.flush(); + } + } + + public void close(Map ackedOffsets) throws IOException { + if (!stores.isEmpty()) { + log.debug("Closing stores."); + for (Map.Entry entry : stores.entrySet()) { + log.debug("Closing storage engine {}", entry.getKey()); + entry.getValue().flush(); + entry.getValue().close(); + } + + Map checkpointOffsets = new HashMap<>(); + for (String storeName : stores.keySet()) { + TopicPartition part = new TopicPartition(storeName, id); + + // only checkpoint the offset to the offsets file if it is persistent; + if (stores.get(storeName).persistent()) { + Long offset = ackedOffsets.get(part); + + if (offset == null) { + // if no record was produced. we need to check the restored offset. + offset = restoredOffsets.get(part); + } + + if (offset != null) { + // store the last offset + 1 (the log position after restoration) + checkpointOffsets.put(part, offset + 1); + } + } + } + + // write the checkpoint file before closing, to indicate clean shutdown + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(this.baseDir, CHECKPOINT_FILE_NAME)); + checkpoint.write(checkpointOffsets); + } + + // release the state directory directoryLock + directoryLock.release(); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java new file mode 100644 index 0000000000000..3efae6518f63d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorTopology.java @@ -0,0 +1,53 @@ +/** + * 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.processor.internals; + +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +public class ProcessorTopology { + + private final List processorNodes; + private final Map sourceByTopics; + + public ProcessorTopology(List processorNodes, + Map sourceByTopics) { + this.processorNodes = Collections.unmodifiableList(processorNodes); + this.sourceByTopics = Collections.unmodifiableMap(sourceByTopics); + } + + public Set sourceTopics() { + return sourceByTopics.keySet(); + } + + public SourceNode source(String topic) { + return sourceByTopics.get(topic); + } + + public Set sources() { + return new HashSet<>(sourceByTopics.values()); + } + + public List processors() { + return processorNodes; + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java new file mode 100644 index 0000000000000..b4b7afe4f03e3 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -0,0 +1,56 @@ +/** + * 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.processor.internals; + +import java.util.PriorityQueue; + +public class PunctuationQueue { + + private PriorityQueue pq = new PriorityQueue<>(); + + public void schedule(PunctuationSchedule sched) { + synchronized (pq) { + pq.add(sched); + } + } + + public void close() { + synchronized (pq) { + pq.clear(); + } + } + + public boolean mayPunctuate(long timestamp, Punctuator punctuator) { + synchronized (pq) { + boolean punctuated = false; + PunctuationSchedule top = pq.peek(); + while (top != null && top.timestamp <= timestamp) { + PunctuationSchedule sched = top; + pq.poll(); + punctuator.punctuate(sched.node(), timestamp); + pq.add(sched.next()); + punctuated = true; + + top = pq.peek(); + } + + return punctuated; + } + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java new file mode 100644 index 0000000000000..dc9a50d32c249 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java @@ -0,0 +1,41 @@ +/** + * 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.processor.internals; + +public class PunctuationSchedule extends Stamped { + + final long interval; + + public PunctuationSchedule(ProcessorNode node, long interval) { + this(node, System.currentTimeMillis(), interval); + } + + public PunctuationSchedule(ProcessorNode node, long time, long interval) { + super(node, time + interval); + this.interval = interval; + } + + public ProcessorNode node() { + return value; + } + + public PunctuationSchedule next() { + return new PunctuationSchedule(value, timestamp, interval); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java new file mode 100644 index 0000000000000..d99e2ae960415 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Punctuator.java @@ -0,0 +1,24 @@ +/** + * 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.processor.internals; + +public interface Punctuator { + + void punctuate(ProcessorNode node, long streamTime); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java new file mode 100644 index 0000000000000..ad2f647a2ac3d --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java @@ -0,0 +1,80 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.Serializer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.HashMap; +import java.util.Map; + +public class RecordCollector { + + private static final Logger log = LoggerFactory.getLogger(RecordCollector.class); + + private final Producer producer; + private final Map offsets; + private final Callback callback = new Callback() { + public void onCompletion(RecordMetadata metadata, Exception exception) { + if (exception == null) { + TopicPartition tp = new TopicPartition(metadata.topic(), metadata.partition()); + offsets.put(tp, metadata.offset()); + } else { + log.error("Error sending record: ", exception); + } + } + }; + + + public RecordCollector(Producer producer) { + this.producer = producer; + this.offsets = new HashMap<>(); + } + + public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { + byte[] keyBytes = keySerializer.serialize(record.topic(), record.key()); + byte[] valBytes = valueSerializer.serialize(record.topic(), record.value()); + this.producer.send(new ProducerRecord<>(record.topic(), keyBytes, valBytes), callback); + } + + public void flush() { + this.producer.flush(); + } + + /** + * Closes this RecordCollector + */ + public void close() { + producer.close(); + } + + /** + * The last ack'd offset from the producer + * + * @return the map from TopicPartition to offset + */ + Map offsets() { + return this.offsets; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java new file mode 100644 index 0000000000000..66f78d2150994 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordQueue.java @@ -0,0 +1,140 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.streams.processor.TimestampExtractor; + +import java.util.ArrayDeque; + +/** + * RecordQueue is a FIFO queue of {@link StampedRecord} (ConsumerRecord + timestamp). It also keeps track of the + * partition timestamp defined as the minimum timestamp of records in its queue; in addition, its partition + * timestamp is monotonically increasing such that once it is advanced, it will not be decremented. + */ +public class RecordQueue { + + private final SourceNode source; + private final TopicPartition partition; + private final ArrayDeque fifoQueue; + private final TimestampTracker> timeTracker; + + private long partitionTime = TimestampTracker.NOT_KNOWN; + + public RecordQueue(TopicPartition partition, SourceNode source) { + this.partition = partition; + this.source = source; + + this.fifoQueue = new ArrayDeque<>(); + this.timeTracker = new MinTimestampTracker<>(); + } + + /** + * Returns the corresponding source node in the topology + * + * @return SourceNode + */ + public SourceNode source() { + return source; + } + + /** + * Returns the partition with which this queue is associated + * + * @return TopicPartition + */ + public TopicPartition partition() { + return partition; + } + + /** + * Add a batch of {@link ConsumerRecord} into the queue + * + * @param rawRecords the raw records + * @param timestampExtractor TimestampExtractor + * @return the size of this queue + */ + public int addRawRecords(Iterable> rawRecords, TimestampExtractor timestampExtractor) { + for (ConsumerRecord rawRecord : rawRecords) { + // deserialize the raw record, extract the timestamp and put into the queue + Object key = source.deserializeKey(rawRecord.topic(), rawRecord.key()); + Object value = source.deserializeValue(rawRecord.topic(), rawRecord.value()); + + ConsumerRecord record = new ConsumerRecord<>(rawRecord.topic(), rawRecord.partition(), rawRecord.offset(), key, value); + long timestamp = timestampExtractor.extract(record); + + StampedRecord stampedRecord = new StampedRecord(record, timestamp); + + fifoQueue.addLast(stampedRecord); + timeTracker.addElement(stampedRecord); + } + + return size(); + } + + /** + * Get the next {@link StampedRecord} from the queue + * + * @return StampedRecord + */ + public StampedRecord poll() { + StampedRecord elem = fifoQueue.pollFirst(); + + if (elem == null) + return null; + + timeTracker.removeElement(elem); + + // only advance the partition timestamp if its currently + // tracked min timestamp has exceeded its value + long timestamp = timeTracker.get(); + + if (timestamp > partitionTime) + partitionTime = timestamp; + + return elem; + } + + /** + * Returns the number of records in the queue + * + * @return the number of records + */ + public int size() { + return fifoQueue.size(); + } + + /** + * Tests if the queue is empty + * + * @return true if the queue is empty, otherwise false + */ + public boolean isEmpty() { + return fifoQueue.isEmpty(); + } + + /** + * Returns the tracked partition timestamp + * + * @return timestamp + */ + public long timestamp() { + return partitionTime; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java new file mode 100644 index 0000000000000..e2d881c9cdea8 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -0,0 +1,64 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.processor.ProcessorContext; + +public class SinkNode extends ProcessorNode { + + private final String topic; + private Serializer keySerializer; + private Serializer valSerializer; + + private ProcessorContext context; + + public SinkNode(String name, String topic, Serializer keySerializer, Serializer valSerializer) { + super(name); + + this.topic = topic; + this.keySerializer = keySerializer; + this.valSerializer = valSerializer; + } + + @Override + public void addChild(ProcessorNode child) { + throw new UnsupportedOperationException("sink node does not allow addChild"); + } + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + this.context = context; + if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerializer(); + if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerializer(); + } + + @Override + public void process(K key, V value) { + // send to all the registered topics + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer); + } + + @Override + public void close() { + // do nothing + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java new file mode 100644 index 0000000000000..fa4afafbbdf27 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -0,0 +1,64 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.processor.ProcessorContext; + +public class SourceNode extends ProcessorNode { + + private Deserializer keyDeserializer; + private Deserializer valDeserializer; + private ProcessorContext context; + + public SourceNode(String name, Deserializer keyDeserializer, Deserializer valDeserializer) { + super(name); + + this.keyDeserializer = keyDeserializer; + this.valDeserializer = valDeserializer; + } + + public K deserializeKey(String topic, byte[] data) { + return keyDeserializer.deserialize(topic, data); + } + + public V deserializeValue(String topic, byte[] data) { + return valDeserializer.deserialize(topic, data); + } + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + this.context = context; + + // if serializers are null, get the default ones from the context + if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keyDeserializer(); + if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueDeserializer(); + } + + @Override + public void process(K key, V value) { + context.forward(key, value); + } + + @Override + public void close() { + // do nothing + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java new file mode 100644 index 0000000000000..4e44667397d06 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/Stamped.java @@ -0,0 +1,38 @@ +/** + * 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.processor.internals; + +public class Stamped implements Comparable { + + public final V value; + public final long timestamp; + + public Stamped(V value, long timestamp) { + this.value = value; + this.timestamp = timestamp; + } + + @Override + public int compareTo(Object other) { + long otherTimestamp = ((Stamped) other).timestamp; + + if (timestamp < otherTimestamp) return -1; + else if (timestamp > otherTimestamp) return 1; + return 0; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java new file mode 100644 index 0000000000000..febd9389fd44c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StampedRecord.java @@ -0,0 +1,52 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +public class StampedRecord extends Stamped> { + + public StampedRecord(ConsumerRecord record, long timestamp) { + super(record, timestamp); + } + + public String topic() { + return value.topic(); + } + + public int partition() { + return value.partition(); + } + + public Object key() { + return value.key(); + } + + public Object value() { + return value.value(); + } + + public long offset() { + return value.offset(); + } + + @Override + public String toString() { + return value.toString() + ", timestamp = " + timestamp; + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java new file mode 100644 index 0000000000000..40fb723577175 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -0,0 +1,352 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.OffsetAndMetadata; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; + +/** + * A StreamTask is associated with a {@link PartitionGroup}, and is assigned to a StreamThread for processing. + */ +public class StreamTask implements Punctuator { + + private static final Logger log = LoggerFactory.getLogger(StreamTask.class); + + private final int id; + private final int maxBufferedSize; + + private final Consumer consumer; + private final PartitionGroup partitionGroup; + private final PartitionGroup.RecordInfo recordInfo = new PartitionGroup.RecordInfo(); + private final PunctuationQueue punctuationQueue; + private final ProcessorContextImpl processorContext; + private final ProcessorTopology topology; + + private final Map consumedOffsets; + private final RecordCollector recordCollector; + private final ProcessorStateManager stateMgr; + + private boolean commitRequested = false; + private boolean commitOffsetNeeded = false; + private StampedRecord currRecord = null; + private ProcessorNode currNode = null; + + /** + * Create {@link StreamTask} with its assigned partitions + * + * @param id the ID of this task + * @param consumer the instance of {@link Consumer} + * @param producer the instance of {@link Producer} + * @param restoreConsumer the instance of {@link Consumer} used when restoring state + * @param partitions the collection of assigned {@link TopicPartition} + * @param topology the instance of {@link ProcessorTopology} + * @param config the {@link StreamingConfig} specified by the user + */ + public StreamTask(int id, + Consumer consumer, + Producer producer, + Consumer restoreConsumer, + Collection partitions, + ProcessorTopology topology, + StreamingConfig config) { + + this.id = id; + this.consumer = consumer; + this.punctuationQueue = new PunctuationQueue(); + this.maxBufferedSize = config.getInt(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + this.topology = topology; + + // create queues for each assigned partition and associate them + // to corresponding source nodes in the processor topology + Map partitionQueues = new HashMap<>(); + + for (TopicPartition partition : partitions) { + SourceNode source = topology.source(partition.topic()); + RecordQueue queue = createRecordQueue(partition, source); + partitionQueues.put(partition, queue); + } + + TimestampExtractor timestampExtractor = config.getConfiguredInstance(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TimestampExtractor.class); + this.partitionGroup = new PartitionGroup(partitionQueues, timestampExtractor); + + // initialize the consumed and produced offset cache + this.consumedOffsets = new HashMap<>(); + + // create the record recordCollector that maintains the produced offsets + this.recordCollector = new RecordCollector(producer); + + log.info("Creating restoration consumer client for stream task [" + id + "]"); + + // create the processor state manager + try { + File stateFile = new File(config.getString(StreamingConfig.STATE_DIR_CONFIG), Integer.toString(id)); + this.stateMgr = new ProcessorStateManager(id, stateFile, restoreConsumer); + } catch (IOException e) { + throw new KafkaException("Error while creating the state manager", e); + } + + // initialize the topology with its own context + this.processorContext = new ProcessorContextImpl(id, this, config, recordCollector, stateMgr, new Metrics()); + + // initialize the task by initializing all its processor nodes in the topology + for (ProcessorNode node : this.topology.processors()) { + this.currNode = node; + try { + node.init(this.processorContext); + } finally { + this.currNode = null; + } + } + + this.processorContext.initialized(); + } + + public int id() { + return id; + } + + public Set partitions() { + return this.partitionGroup.partitions(); + } + + /** + * Adds records to queues + * + * @param partition the partition + * @param records the records + */ + @SuppressWarnings("unchecked") + public void addRecords(TopicPartition partition, Iterable> records) { + int queueSize = partitionGroup.addRawRecords(partition, records); + + // if after adding these records, its partition queue's buffered size has been + // increased beyond the threshold, we can then pause the consumption for this partition + if (queueSize > this.maxBufferedSize) { + consumer.pause(partition); + } + } + + /** + * Process one record + * + * @return number of records left in the buffer of this task's partition group after the processing is done + */ + @SuppressWarnings("unchecked") + public int process() { + synchronized (this) { + // get the next record to process + StampedRecord record = partitionGroup.nextRecord(recordInfo); + + // if there is no record to process, return immediately + if (record == null) + return 0; + + try { + // process the record by passing to the source node of the topology + this.currRecord = record; + this.currNode = recordInfo.node(); + TopicPartition partition = recordInfo.partition(); + + log.debug("Start processing one record [" + currRecord + "]"); + + this.currNode.process(currRecord.key(), currRecord.value()); + + log.debug("Completed processing one record [" + currRecord + "]"); + + // update the consumed offset map after processing is done + consumedOffsets.put(partition, currRecord.offset()); + commitOffsetNeeded = true; + + // after processing this record, if its partition queue's buffered size has been + // decreased to the threshold, we can then resume the consumption on this partition + if (partitionGroup.numBuffered(partition) == this.maxBufferedSize) { + consumer.resume(partition); + } + } finally { + this.currRecord = null; + this.currNode = null; + } + + return partitionGroup.numBuffered(); + } + } + + /** + * Possibly trigger registered punctuation functions if + * current time has reached the defined stamp + * + * @param timestamp + */ + public boolean maybePunctuate(long timestamp) { + return punctuationQueue.mayPunctuate(timestamp, this); + } + + @Override + public void punctuate(ProcessorNode node, long timestamp) { + if (currNode != null) + throw new IllegalStateException("Current node is not null"); + + currNode = node; + try { + node.processor().punctuate(timestamp); + } finally { + currNode = null; + } + } + + public StampedRecord record() { + return this.currRecord; + } + + public ProcessorNode node() { + return this.currNode; + } + + public ProcessorTopology topology() { + return this.topology; + } + + /** + * Commit the current task state + */ + public void commit() { + // 1) flush produced records in the downstream and change logs of local states + recordCollector.flush(); + + // 2) flush local state + stateMgr.flush(); + + // 3) commit consumed offsets if it is dirty already + if (commitOffsetNeeded) { + Map consumedOffsetsAndMetadata = new HashMap<>(consumedOffsets.size()); + for (Map.Entry entry : consumedOffsets.entrySet()) { + consumedOffsetsAndMetadata.put(entry.getKey(), new OffsetAndMetadata(entry.getValue())); + } + consumer.commitSync(consumedOffsetsAndMetadata); + commitOffsetNeeded = false; + } + + commitRequested = false; + } + + /** + * Whether or not a request has been made to commit the current state + */ + public boolean commitNeeded() { + return this.commitRequested; + } + + /** + * Request committing the current task's state + */ + public void needCommit() { + this.commitRequested = true; + } + + /** + * Schedules a punctuation for the processor + * + * @param interval the interval in milliseconds + */ + public void schedule(long interval) { + if (currNode == null) + throw new IllegalStateException("Current node is null"); + + punctuationQueue.schedule(new PunctuationSchedule(currNode, interval)); + } + + public void close() { + this.partitionGroup.close(); + this.consumedOffsets.clear(); + + // close the processors + // make sure close() is called for each node even when there is a RuntimeException + RuntimeException exception = null; + for (ProcessorNode node : this.topology.processors()) { + currNode = node; + try { + node.close(); + } catch (RuntimeException e) { + exception = e; + } finally { + currNode = null; + } + } + + if (exception != null) + throw exception; + + try { + stateMgr.close(recordCollector.offsets()); + } catch (IOException e) { + throw new KafkaException("Error while closing the state manager in processor context", e); + } + } + + private RecordQueue createRecordQueue(TopicPartition partition, SourceNode source) { + return new RecordQueue(partition, source); + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode thisNode = currNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + + public ProcessorContext context() { + return processorContext; + } + +} 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 new file mode 100644 index 0000000000000..f37903f20527f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -0,0 +1,477 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Count; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileLock; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +public class StreamThread extends Thread { + + private static final Logger log = LoggerFactory.getLogger(StreamThread.class); + private static AtomicInteger nextThreadNumber = new AtomicInteger(1); + + private final AtomicBoolean running; + + protected final StreamingConfig config; + protected final TopologyBuilder builder; + protected final Producer producer; + protected final Consumer consumer; + protected final Consumer restoreConsumer; + + private final Map tasks; + private final Time time; + private final File stateDir; + private final long pollTimeMs; + private final long cleanTimeMs; + private final long commitTimeMs; + private final long totalRecordsToProcess; + private final StreamingMetrics metrics; + + private long lastClean; + private long lastCommit; + private long recordsProcessed; + + final ConsumerRebalanceListener rebalanceListener = new ConsumerRebalanceListener() { + @Override + public void onPartitionsAssigned(Collection assignment) { + addPartitions(assignment); + lastClean = time.milliseconds(); // start the cleaning cycle + } + + @Override + public void onPartitionsRevoked(Collection assignment) { + commitAll(); + removePartitions(); + lastClean = Long.MAX_VALUE; // stop the cleaning cycle until partitions are assigned + } + }; + + public StreamThread(TopologyBuilder builder, StreamingConfig config) throws Exception { + this(builder, config, null , null, null, new SystemTime()); + } + + StreamThread(TopologyBuilder builder, StreamingConfig config, + Producer producer, + Consumer consumer, + Consumer restoreConsumer, + Time time) throws Exception { + super("StreamThread-" + nextThreadNumber.getAndIncrement()); + + this.config = config; + this.builder = builder; + + // set the producer and consumer clients + this.producer = (producer != null) ? producer : createProducer(); + this.consumer = (consumer != null) ? consumer : createConsumer(); + this.restoreConsumer = (restoreConsumer != null) ? restoreConsumer : createRestoreConsumer(); + + // initialize the task list + this.tasks = new HashMap<>(); + + // read in task specific config values + this.stateDir = new File(this.config.getString(StreamingConfig.STATE_DIR_CONFIG)); + this.stateDir.mkdir(); + this.pollTimeMs = config.getLong(StreamingConfig.POLL_MS_CONFIG); + this.commitTimeMs = config.getLong(StreamingConfig.COMMIT_INTERVAL_MS_CONFIG); + this.cleanTimeMs = config.getLong(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG); + this.totalRecordsToProcess = config.getLong(StreamingConfig.TOTAL_RECORDS_TO_PROCESS); + + this.lastClean = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment + this.lastCommit = time.milliseconds(); + this.recordsProcessed = 0; + this.time = time; + + this.metrics = new StreamingMetrics(); + + this.running = new AtomicBoolean(true); + } + + private Producer createProducer() { + log.info("Creating producer client for stream thread [" + this.getName() + "]"); + return new KafkaProducer<>(config.getProducerConfigs(), + new ByteArraySerializer(), + new ByteArraySerializer()); + } + + private Consumer createConsumer() { + log.info("Creating consumer client for stream thread [" + this.getName() + "]"); + return new KafkaConsumer<>(config.getConsumerConfigs(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + } + + private Consumer createRestoreConsumer() { + log.info("Creating restore consumer client for stream thread [" + this.getName() + "]"); + return new KafkaConsumer<>(config.getConsumerConfigs(), + new ByteArrayDeserializer(), + new ByteArrayDeserializer()); + } + + /** + * Execute the stream processors + */ + @Override + public void run() { + log.info("Starting stream thread [" + this.getName() + "]"); + + try { + runLoop(); + } catch (RuntimeException e) { + log.error("Uncaught error during processing in thread [" + this.getName() + "]: ", e); + throw e; + } finally { + shutdown(); + } + } + + /** + * Shutdown this streaming thread. + */ + public void close() { + running.set(false); + } + + public Map tasks() { + return Collections.unmodifiableMap(tasks); + } + + private void shutdown() { + log.info("Shutting down stream thread [" + this.getName() + "]"); + + // Exceptions should not prevent this call from going through all shutdown steps. + try { + commitAll(); + } catch (Throwable e) { + // already logged in commitAll() + } + try { + producer.close(); + } catch (Throwable e) { + log.error("Failed to close producer in thread [" + this.getName() + "]: ", e); + } + try { + consumer.close(); + } catch (Throwable e) { + log.error("Failed to close consumer in thread [" + this.getName() + "]: ", e); + } + try { + restoreConsumer.close(); + } catch (Throwable e) { + log.error("Failed to close restore consumer in thread [" + this.getName() + "]: ", e); + } + try { + removePartitions(); + } catch (Throwable e) { + // already logged in removePartition() + } + + log.info("Stream thread shutdown complete [" + this.getName() + "]"); + } + + private void runLoop() { + try { + int totalNumBuffered = 0; + + consumer.subscribe(new ArrayList<>(builder.sourceTopics()), rebalanceListener); + + while (stillRunning()) { + long startPoll = time.milliseconds(); + + // try to fetch some records if necessary + ConsumerRecords records = consumer.poll(totalNumBuffered == 0 ? this.pollTimeMs : 0); + + for (StreamTask task : tasks.values()) { + for (TopicPartition partition : task.partitions()) { + task.addRecords(partition, records.records(partition)); + } + } + + long endPoll = time.milliseconds(); + metrics.pollTimeSensor.record(endPoll - startPoll); + + // try to process one record from each task + totalNumBuffered = 0; + + for (StreamTask task : tasks.values()) { + long startProcess = time.milliseconds(); + + totalNumBuffered += task.process(); + + metrics.processTimeSensor.record(time.milliseconds() - startProcess); + } + + maybePunctuate(); + maybeClean(); + maybeCommit(); + } + } catch (Exception e) { + throw new KafkaException(e); + } + } + + private boolean stillRunning() { + if (!running.get()) { + log.debug("Shutting down at user request."); + return false; + } + + if (totalRecordsToProcess >= 0 && recordsProcessed >= totalRecordsToProcess) { + log.debug("Shutting down as we've reached the user configured limit of {} records to process.", totalRecordsToProcess); + return false; + } + + return true; + } + + private void maybePunctuate() { + for (StreamTask task : tasks.values()) { + try { + long now = time.milliseconds(); + + if (task.maybePunctuate(now)) + metrics.punctuateTimeSensor.record(time.milliseconds() - now); + + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + } + } + + protected void maybeCommit() { + long now = time.milliseconds(); + + if (commitTimeMs >= 0 && lastCommit + commitTimeMs < now) { + log.trace("Committing processor instances because the commit interval has elapsed."); + + commitAll(); + lastCommit = now; + } else { + for (StreamTask task : tasks.values()) { + try { + if (task.commitNeeded()) + commitOne(task, time.milliseconds()); + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + } + } + } + + /** + * Commit the states of all its tasks + */ + private void commitAll() { + for (StreamTask task : tasks.values()) { + try { + commitOne(task, time.milliseconds()); + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + } + } + + /** + * Commit the state of a task + */ + private void commitOne(StreamTask task, long now) { + try { + task.commit(); + } catch (Exception e) { + log.error("Failed to commit task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + + metrics.commitTimeSensor.record(time.milliseconds() - now); + } + + /** + * Cleanup any states of the tasks that have been removed from this thread + */ + protected void maybeClean() { + long now = time.milliseconds(); + + if (now > lastClean + cleanTimeMs) { + File[] stateDirs = stateDir.listFiles(); + if (stateDirs != null) { + for (File dir : stateDirs) { + try { + Integer id = Integer.parseInt(dir.getName()); + + // try to acquire the exclusive lock on the state directory + FileLock directoryLock = null; + try { + directoryLock = ProcessorStateManager.lockStateDirectory(dir); + if (directoryLock != null) { + log.info("Deleting obsolete state directory {} after delayed {} ms.", dir.getAbsolutePath(), cleanTimeMs); + Utils.delete(dir); + } + } catch (IOException e) { + log.error("Failed to lock the state directory due to an unexpected exception", e); + } finally { + if (directoryLock != null) { + try { + directoryLock.release(); + } catch (IOException e) { + log.error("Failed to release the state directory lock"); + } + } + } + } catch (NumberFormatException e) { + // there may be some unknown files that sits in the same directory, + // we should ignore these files instead trying to delete them as well + } + } + } + + lastClean = now; + } + } + + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + metrics.taskCreationSensor.record(); + + return new StreamTask(id, consumer, producer, restoreConsumer, partitionsForTask, builder.build(), config); + } + + private void addPartitions(Collection assignment) { + HashSet partitions = new HashSet<>(assignment); + + // TODO: change this hard-coded co-partitioning behavior + for (TopicPartition partition : partitions) { + final Integer id = partition.partition(); + StreamTask task = tasks.get(id); + if (task == null) { + // get the partitions for the task + HashSet partitionsForTask = new HashSet<>(); + for (TopicPartition part : partitions) + if (part.partition() == id) + partitionsForTask.add(part); + + // create the task + try { + task = createStreamTask(id, partitionsForTask); + } catch (Exception e) { + log.error("Failed to create a task #" + id + " in thread [" + this.getName() + "]: ", e); + throw e; + } + tasks.put(id, task); + } + } + + lastClean = time.milliseconds(); + } + + private void removePartitions() { + + // TODO: change this clearing tasks behavior + for (StreamTask task : tasks.values()) { + log.info("Removing task {}", task.id()); + try { + task.close(); + } catch (Exception e) { + log.error("Failed to close a task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; + } + metrics.taskDestructionSensor.record(); + } + tasks.clear(); + } + + private class StreamingMetrics { + final Metrics metrics; + + final Sensor commitTimeSensor; + final Sensor pollTimeSensor; + final Sensor processTimeSensor; + final Sensor punctuateTimeSensor; + final Sensor taskCreationSensor; + final Sensor taskDestructionSensor; + + public StreamingMetrics() { + String metricGrpName = "streaming-metrics"; + + this.metrics = new Metrics(); + Map metricTags = new LinkedHashMap(); + metricTags.put("client-id", config.getString(StreamingConfig.CLIENT_ID_CONFIG) + "-" + getName()); + + this.commitTimeSensor = metrics.sensor("commit-time"); + this.commitTimeSensor.add(new MetricName("commit-time-avg", metricGrpName, "The average commit time in ms", metricTags), new Avg()); + this.commitTimeSensor.add(new MetricName("commit-time-max", metricGrpName, "The maximum commit time in ms", metricTags), new Max()); + this.commitTimeSensor.add(new MetricName("commit-calls-rate", metricGrpName, "The average per-second number of commit calls", metricTags), new Rate(new Count())); + + this.pollTimeSensor = metrics.sensor("poll-time"); + this.pollTimeSensor.add(new MetricName("poll-time-avg", metricGrpName, "The average poll time in ms", metricTags), new Avg()); + this.pollTimeSensor.add(new MetricName("poll-time-max", metricGrpName, "The maximum poll time in ms", metricTags), new Max()); + this.pollTimeSensor.add(new MetricName("poll-calls-rate", metricGrpName, "The average per-second number of record-poll calls", metricTags), new Rate(new Count())); + + this.processTimeSensor = metrics.sensor("process-time"); + this.processTimeSensor.add(new MetricName("process-time-avg-ms", metricGrpName, "The average process time in ms", metricTags), new Avg()); + this.processTimeSensor.add(new MetricName("process-time-max-ms", metricGrpName, "The maximum process time in ms", metricTags), new Max()); + this.processTimeSensor.add(new MetricName("process-calls-rate", metricGrpName, "The average per-second number of process calls", metricTags), new Rate(new Count())); + + this.punctuateTimeSensor = metrics.sensor("punctuate-time"); + this.punctuateTimeSensor.add(new MetricName("punctuate-time-avg", metricGrpName, "The average punctuate time in ms", metricTags), new Avg()); + this.punctuateTimeSensor.add(new MetricName("punctuate-time-max", metricGrpName, "The maximum punctuate time in ms", metricTags), new Max()); + this.punctuateTimeSensor.add(new MetricName("punctuate-calls-rate", metricGrpName, "The average per-second number of punctuate calls", metricTags), new Rate(new Count())); + + this.taskCreationSensor = metrics.sensor("task-creation"); + this.taskCreationSensor.add(new MetricName("task-creation-rate", metricGrpName, "The average per-second number of newly created tasks", metricTags), new Rate(new Count())); + + this.taskDestructionSensor = metrics.sensor("task-destruction"); + this.taskDestructionSensor.add(new MetricName("task-destruction-rate", metricGrpName, "The average per-second number of destructed tasks", metricTags), new Rate(new Count())); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java new file mode 100644 index 0000000000000..d8a012a1bdcba --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/TimestampTracker.java @@ -0,0 +1,58 @@ +/** + * 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.processor.internals; + +/** + * TimestampTracker is a helper class for a sliding window implementation. + * It is assumed that stamped elements are added or removed in a FIFO manner. + * It maintains the timestamp, such as the min timestamp, the max timestamp, etc. + * of stamped elements that were added but not yet removed. + */ +public interface TimestampTracker { + + static final long NOT_KNOWN = -1L; + + /** + * Adds a stamped elements to this tracker. + * + * @param elem the added element + */ + void addElement(Stamped elem); + + /** + * Removed a stamped elements to this tracker. + * + * @param elem the removed element + */ + void removeElement(Stamped elem); + + /** + * Returns the current tracked timestamp + * + * @return timestamp, or {@link #NOT_KNOWN} when empty + */ + long get(); + + /** + * Returns the size of internal structure. The meaning of "size" depends on the implementation. + * + * @return size + */ + int size(); + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Entry.java b/streams/src/main/java/org/apache/kafka/streams/state/Entry.java new file mode 100644 index 0000000000000..183b691c1acf6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/Entry.java @@ -0,0 +1,42 @@ +/** + * 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.state; + +public class Entry { + + private final K key; + private final V value; + + public Entry(K key, V value) { + this.key = key; + this.value = value; + } + + public K key() { + return key; + } + + public V value() { + return value; + } + + public String toString() { + return "Entry(" + key() + ", " + value() + ")"; + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java new file mode 100644 index 0000000000000..e9aaa20265a53 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/InMemoryKeyValueStore.java @@ -0,0 +1,145 @@ +/** + * 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.state; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Time; + +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.NavigableMap; +import java.util.TreeMap; + +/** + * An in-memory key-value store based on a TreeMap + * + * @param The key type + * @param The value type + */ +public class InMemoryKeyValueStore extends MeteredKeyValueStore { + + public InMemoryKeyValueStore(String name, ProcessorContext context) { + this(name, context, new SystemTime()); + } + + public InMemoryKeyValueStore(String name, ProcessorContext context, Time time) { + super(name, new MemoryStore(name, context), context, "kafka-streams", time); + } + + private static class MemoryStore implements KeyValueStore { + + private final String name; + private final NavigableMap map; + private final ProcessorContext context; + + @SuppressWarnings("unchecked") + public MemoryStore(String name, ProcessorContext context) { + super(); + this.name = name; + this.map = new TreeMap<>(); + this.context = context; + } + + @Override + public String name() { + return this.name; + } + + @Override + public boolean persistent() { + return false; + } + + @Override + public V get(K key) { + return this.map.get(key); + } + + @Override + public void put(K key, V value) { + this.map.put(key, value); + } + + @Override + public void putAll(List> entries) { + for (Entry entry : entries) + put(entry.key(), entry.value()); + } + + @Override + public V delete(K key) { + return this.map.remove(key); + } + + @Override + public KeyValueIterator range(K from, K to) { + return new MemoryStoreIterator(this.map.subMap(from, true, to, false).entrySet().iterator()); + } + + @Override + public KeyValueIterator all() { + return new MemoryStoreIterator(this.map.entrySet().iterator()); + } + + @Override + public void flush() { + // do-nothing since it is in-memory + } + + public void restore() { + // this should not happen since it is in-memory, hence no state to load from disk + throw new IllegalStateException("This should not happen"); + } + + @Override + public void close() { + // do-nothing + } + + private static class MemoryStoreIterator implements KeyValueIterator { + private final Iterator> iter; + + public MemoryStoreIterator(Iterator> iter) { + this.iter = iter; + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Entry next() { + Map.Entry entry = iter.next(); + return new Entry<>(entry.getKey(), entry.getValue()); + } + + @Override + public void remove() { + iter.remove(); + } + + @Override + public void close() { + } + + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java new file mode 100644 index 0000000000000..0fbd4ae536f1f --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueIterator.java @@ -0,0 +1,29 @@ +/** + * 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.state; + +import java.io.Closeable; +import java.util.Iterator; + +public interface KeyValueIterator extends Iterator>, Closeable { + + @Override + public void close(); +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java new file mode 100644 index 0000000000000..e4faed1efc576 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java @@ -0,0 +1,86 @@ +/** + * 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.state; + +import org.apache.kafka.streams.processor.StateStore; + +import java.util.List; + +/** + * A key-value store that supports put/get/delete and range queries. + * + * @param The key type + * @param The value type + */ +public interface KeyValueStore extends StateStore { + + /** + * Get the value corresponding to this key + * + * @param key The key to fetch + * @return The value or null if no value is found. + * @throws NullPointerException If null is used for key. + */ + abstract public V get(K key); + + /** + * Update the value associated with this key + * + * @param key They key to associate the value to + * @param value The value + * @throws NullPointerException If null is used for key or value. + */ + abstract public void put(K key, V value); + + /** + * Update all the given key/value pairs + * + * @param entries A list of entries to put into the store. + * @throws NullPointerException If null is used for any key or value. + */ + abstract public void putAll(List> entries); + + /** + * Delete the value from the store (if there is one) + * + * @param key The key + * @return The old value or null if there is no such key. + * @throws NullPointerException If null is used for key. + */ + abstract public V delete(K key); + + /** + * Get an iterator over a given range of keys. This iterator MUST be closed after use. + * + * @param from The first key that could be in the range + * @param to The last key that could be in the range + * @return The iterator for this range. + * @throws NullPointerException If null is used for from or to. + */ + abstract public KeyValueIterator range(K from, K to); + + /** + * Return an iterator over all keys in the database. This iterator MUST be closed after use. + * + * @return An iterator of all key/value pairs in the store. + */ + abstract public KeyValueIterator all(); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java new file mode 100644 index 0000000000000..018f1c621c64e --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/MeteredKeyValueStore.java @@ -0,0 +1,273 @@ +/** + * 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.state; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.RestoreFunc; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.MetricName; +import org.apache.kafka.common.metrics.MeasurableStat; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.metrics.stats.Avg; +import org.apache.kafka.common.metrics.stats.Count; +import org.apache.kafka.common.metrics.stats.Max; +import org.apache.kafka.common.metrics.stats.Rate; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.utils.Time; +import org.apache.kafka.streams.processor.internals.ProcessorContextImpl; +import org.apache.kafka.streams.processor.internals.RecordCollector; + +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +public class MeteredKeyValueStore implements KeyValueStore { + + protected final KeyValueStore inner; + + private final Time time; + private final String group; + private final Sensor putTime; + private final Sensor getTime; + private final Sensor deleteTime; + private final Sensor putAllTime; + private final Sensor allTime; + private final Sensor rangeTime; + private final Sensor flushTime; + private final Sensor restoreTime; + private final Metrics metrics; + + private final String topic; + private final int partition; + private final Set dirty; + private final int maxDirty; + private final ProcessorContext context; + + // always wrap the logged store with the metered store + public MeteredKeyValueStore(final String name, final KeyValueStore inner, ProcessorContext context, String group, Time time) { + this.inner = inner; + + this.time = time; + this.group = group; + this.metrics = context.metrics(); + this.putTime = createSensor(name, "put"); + this.getTime = createSensor(name, "get"); + this.deleteTime = createSensor(name, "delete"); + this.putAllTime = createSensor(name, "put-all"); + this.allTime = createSensor(name, "all"); + this.rangeTime = createSensor(name, "range"); + this.flushTime = createSensor(name, "flush"); + this.restoreTime = createSensor(name, "restore"); + + this.topic = name; + this.partition = context.id(); + + this.context = context; + + this.dirty = new HashSet(); + this.maxDirty = 100; // TODO: this needs to be configurable + + // register and possibly restore the state from the logs + long startNs = time.nanoseconds(); + try { + final Deserializer keyDeserializer = (Deserializer) context.keyDeserializer(); + final Deserializer valDeserializer = (Deserializer) context.valueDeserializer(); + + context.register(this, new RestoreFunc() { + @Override + public void apply(byte[] key, byte[] value) { + inner.put(keyDeserializer.deserialize(topic, key), + valDeserializer.deserialize(topic, value)); + } + }); + } finally { + recordLatency(this.restoreTime, startNs, time.nanoseconds()); + } + } + + private Sensor createSensor(String storeName, String operation) { + Sensor parent = metrics.sensor(operation); + addLatencyMetrics(parent, operation); + Sensor sensor = metrics.sensor(storeName + "- " + operation, parent); + addLatencyMetrics(sensor, operation, "store-name", storeName); + return sensor; + } + + private void addLatencyMetrics(Sensor sensor, String opName, String... kvs) { + maybeAddMetric(sensor, new MetricName(opName + "-avg-latency-ms", group, "The average latency in milliseconds of the key-value store operation.", kvs), new Avg()); + maybeAddMetric(sensor, new MetricName(opName + "-max-latency-ms", group, "The max latency in milliseconds of the key-value store operation.", kvs), new Max()); + maybeAddMetric(sensor, new MetricName(opName + "-qps", group, "The average number of occurance of the given key-value store operation per second.", kvs), new Rate(new Count())); + } + + private void maybeAddMetric(Sensor sensor, MetricName name, MeasurableStat stat) { + if (!metrics.metrics().containsKey(name)) + sensor.add(name, stat); + } + + @Override + public String name() { + return inner.name(); + } + + @Override + public boolean persistent() { + return inner.persistent(); + } + + @Override + public V get(K key) { + long startNs = time.nanoseconds(); + try { + return this.inner.get(key); + } finally { + recordLatency(this.getTime, startNs, time.nanoseconds()); + } + } + + @Override + public void put(K key, V value) { + long startNs = time.nanoseconds(); + try { + this.inner.put(key, value); + + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + logChange(); + } finally { + recordLatency(this.putTime, startNs, time.nanoseconds()); + } + } + + @Override + public void putAll(List> entries) { + long startNs = time.nanoseconds(); + try { + this.inner.putAll(entries); + + for (Entry entry : entries) { + this.dirty.add(entry.key()); + } + + if (this.dirty.size() > this.maxDirty) + logChange(); + } finally { + recordLatency(this.putAllTime, startNs, time.nanoseconds()); + } + } + + @Override + public V delete(K key) { + long startNs = time.nanoseconds(); + try { + V value = this.inner.delete(key); + + this.dirty.add(key); + if (this.dirty.size() > this.maxDirty) + logChange(); + + return value; + } finally { + recordLatency(this.deleteTime, startNs, time.nanoseconds()); + } + } + + @Override + public KeyValueIterator range(K from, K to) { + return new MeteredKeyValueIterator(this.inner.range(from, to), this.rangeTime); + } + + @Override + public KeyValueIterator all() { + return new MeteredKeyValueIterator(this.inner.all(), this.allTime); + } + + @Override + public void close() { + inner.close(); + } + + @Override + public void flush() { + long startNs = time.nanoseconds(); + try { + this.inner.flush(); + logChange(); + } finally { + recordLatency(this.flushTime, startNs, time.nanoseconds()); + } + } + + private void logChange() { + RecordCollector collector = ((ProcessorContextImpl) context).recordCollector(); + Serializer keySerializer = (Serializer) context.keySerializer(); + Serializer valueSerializer = (Serializer) context.valueSerializer(); + + if (collector != null) { + for (K k : this.dirty) { + V v = this.inner.get(k); + collector.send(new ProducerRecord<>(this.topic, this.partition, k, v), keySerializer, valueSerializer); + } + this.dirty.clear(); + } + } + + private void recordLatency(Sensor sensor, long startNs, long endNs) { + sensor.record((endNs - startNs) / 1000000, endNs); + } + + private class MeteredKeyValueIterator implements KeyValueIterator { + + private final KeyValueIterator iter; + private final Sensor sensor; + private final long startNs; + + public MeteredKeyValueIterator(KeyValueIterator iter, Sensor sensor) { + this.iter = iter; + this.sensor = sensor; + this.startNs = time.nanoseconds(); + } + + @Override + public boolean hasNext() { + return iter.hasNext(); + } + + @Override + public Entry next() { + return iter.next(); + } + + @Override + public void remove() { + iter.remove(); + } + + @Override + public void close() { + try { + iter.close(); + } finally { + recordLatency(this.sensor, this.startNs, time.nanoseconds()); + } + } + + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java b/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java new file mode 100644 index 0000000000000..e04de68afde8c --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/OffsetCheckpoint.java @@ -0,0 +1,172 @@ +/** + * 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.state; + +import org.apache.kafka.common.TopicPartition; + +import java.io.BufferedReader; +import java.io.BufferedWriter; +import java.io.EOFException; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.OutputStreamWriter; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; + +/** + * This class saves out a map of topic/partition=>offsets to a file. The format of the file is UTF-8 text containing the following: + *

+ *   <version>
+ *   <n>
+ *   <topic_name_1> <partition_1> <offset_1>
+ *   .
+ *   .
+ *   .
+ *   <topic_name_n> <partition_n> <offset_n>
+ * 
+ * The first line contains a number designating the format version (currently 0), the get line contains + * a number giving the total number of offsets. Each successive line gives a topic/partition/offset triple + * separated by spaces. + */ +public class OffsetCheckpoint { + + private static final int VERSION = 0; + + private final File file; + private final Object lock; + + public OffsetCheckpoint(File file) throws IOException { + new File(file + ".tmp").delete(); // try to delete any existing temp files for cleanliness + this.file = file; + this.lock = new Object(); + } + + public void write(Map offsets) throws IOException { + synchronized (lock) { + // write to temp file and then swap with the existing file + File temp = new File(file.getAbsolutePath() + ".tmp"); + + FileOutputStream fileOutputStream = new FileOutputStream(temp); + BufferedWriter writer = new BufferedWriter(new OutputStreamWriter(fileOutputStream)); + try { + writeIntLine(writer, VERSION); + writeIntLine(writer, offsets.size()); + + // write the entries + for (Map.Entry entry : offsets.entrySet()) + writeEntry(writer, entry.getKey(), entry.getValue()); + + // flush the buffer and then fsync the underlying file + writer.flush(); + fileOutputStream.getFD().sync(); + } finally { + writer.close(); + } + + // swap new offset checkpoint file with previous one + if (!temp.renameTo(file)) { + // renameTo() fails on Windows if the destination file exists. + file.delete(); + if (!temp.renameTo(file)) + throw new IOException(String.format("File rename from %s to %s failed.", + temp.getAbsolutePath(), + file.getAbsolutePath())); + } + } + } + + private void writeIntLine(BufferedWriter writer, int number) throws IOException { + writer.write(Integer.toString(number)); + writer.newLine(); + } + + private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) throws IOException { + writer.write(part.topic()); + writer.write(' '); + writer.write(Integer.toString(part.partition())); + writer.write(' '); + writer.write(Long.toString(offset)); + writer.newLine(); + } + + public Map read() throws IOException { + synchronized (lock) { + BufferedReader reader = null; + try { + reader = new BufferedReader(new FileReader(file)); + } catch (FileNotFoundException e) { + return Collections.emptyMap(); + } + + try { + int version = readInt(reader); + switch (version) { + case 0: + int expectedSize = readInt(reader); + Map offsets = new HashMap(); + String line = reader.readLine(); + while (line != null) { + String[] pieces = line.split("\\s+"); + if (pieces.length != 3) + throw new IOException(String.format("Malformed line in offset checkpoint file: '%s'.", + line)); + + String topic = pieces[0]; + int partition = Integer.parseInt(pieces[1]); + long offset = Long.parseLong(pieces[2]); + offsets.put(new TopicPartition(topic, partition), offset); + line = reader.readLine(); + } + if (offsets.size() != expectedSize) + throw new IOException(String.format("Expected %d entries but found only %d", + expectedSize, + offsets.size())); + return offsets; + + default: + throw new IllegalArgumentException("Unknown offset checkpoint version: " + version); + } + } finally { + if (reader != null) + reader.close(); + } + } + } + + private int readInt(BufferedReader reader) throws IOException { + String line = reader.readLine(); + if (line == null) + throw new EOFException("File ended prematurely."); + int val = Integer.parseInt(line); + return val; + } + + public void delete() throws IOException { + file.delete(); + } + + @Override + public String toString() { + return this.file.getAbsolutePath(); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java new file mode 100644 index 0000000000000..e0962a206ad71 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/RocksDBKeyValueStore.java @@ -0,0 +1,276 @@ +/** + * 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.state; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.utils.SystemTime; + +import org.apache.kafka.common.utils.Time; +import org.rocksdb.BlockBasedTableConfig; +import org.rocksdb.CompactionStyle; +import org.rocksdb.CompressionType; +import org.rocksdb.FlushOptions; +import org.rocksdb.Options; +import org.rocksdb.RocksDB; +import org.rocksdb.RocksDBException; +import org.rocksdb.RocksIterator; +import org.rocksdb.WriteOptions; + +import java.io.File; +import java.util.Comparator; +import java.util.List; +import java.util.NoSuchElementException; + +public class RocksDBKeyValueStore extends MeteredKeyValueStore { + + public RocksDBKeyValueStore(String name, ProcessorContext context) { + this(name, context, new SystemTime()); + } + + public RocksDBKeyValueStore(String name, ProcessorContext context, Time time) { + super(name, new RocksDBStore(name, context), context, "kafka-streams", time); + } + + private static class RocksDBStore implements KeyValueStore { + + private static final int TTL_NOT_USED = -1; + + // TODO: these values should be configurable + private static final long WRITE_BUFFER_SIZE = 32 * 1024 * 1024L; + private static final long BLOCK_CACHE_SIZE = 100 * 1024 * 1024L; + private static final long BLOCK_SIZE = 4096L; + private static final int TTL_SECONDS = TTL_NOT_USED; + private static final int MAX_WRITE_BUFFERS = 3; + private static final CompressionType COMPRESSION_TYPE = CompressionType.NO_COMPRESSION; + private static final CompactionStyle COMPACTION_STYLE = CompactionStyle.UNIVERSAL; + private static final String DB_FILE_DIR = "rocksdb"; + + private final String topic; + private final int partition; + private final ProcessorContext context; + + private final Options options; + private final WriteOptions wOptions; + private final FlushOptions fOptions; + + private final String dbName; + private final String dirName; + + private RocksDB db; + + @SuppressWarnings("unchecked") + public RocksDBStore(String name, ProcessorContext context) { + this.topic = name; + this.partition = context.id(); + this.context = context; + + // initialize the rocksdb options + BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); + tableConfig.setBlockCacheSize(BLOCK_CACHE_SIZE); + tableConfig.setBlockSize(BLOCK_SIZE); + + options = new Options(); + options.setTableFormatConfig(tableConfig); + options.setWriteBufferSize(WRITE_BUFFER_SIZE); + options.setCompressionType(COMPRESSION_TYPE); + options.setCompactionStyle(COMPACTION_STYLE); + options.setMaxWriteBufferNumber(MAX_WRITE_BUFFERS); + options.setCreateIfMissing(true); + options.setErrorIfExists(false); + + wOptions = new WriteOptions(); + wOptions.setDisableWAL(true); + + fOptions = new FlushOptions(); + fOptions.setWaitForFlush(true); + + dbName = this.topic + "." + this.partition; + dirName = this.context.stateDir() + File.separator + DB_FILE_DIR; + + db = openDB(new File(dirName, dbName), this.options, TTL_SECONDS); + } + + private RocksDB openDB(File dir, Options options, int ttl) { + try { + if (ttl == TTL_NOT_USED) { + return RocksDB.open(options, dir.toString()); + } else { + throw new KafkaException("Change log is not supported for store " + this.topic + " since it is TTL based."); + // TODO: support TTL with change log? + // return TtlDB.open(options, dir.toString(), ttl, false); + } + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error opening store " + this.topic + " at location " + dir.toString(), e); + } + } + + @Override + public String name() { + return this.topic; + } + + @Override + public boolean persistent() { + return false; + } + + @Override + public byte[] get(byte[] key) { + try { + return this.db.get(key); + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error while executing get " + key.toString() + " from store " + this.topic, e); + } + } + + @Override + public void put(byte[] key, byte[] value) { + try { + if (value == null) { + db.remove(wOptions, key); + } else { + db.put(wOptions, key, value); + } + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error while executing put " + key.toString() + " from store " + this.topic, e); + } + } + + @Override + public void putAll(List> entries) { + for (Entry entry : entries) + put(entry.key(), entry.value()); + } + + @Override + public byte[] delete(byte[] key) { + byte[] value = get(key); + put(key, null); + return value; + } + + @Override + public KeyValueIterator range(byte[] from, byte[] to) { + return new RocksDBRangeIterator(db.newIterator(), from, to); + } + + @Override + public KeyValueIterator all() { + RocksIterator innerIter = db.newIterator(); + innerIter.seekToFirst(); + return new RocksDbIterator(innerIter); + } + + @Override + public void flush() { + try { + db.flush(fOptions); + } catch (RocksDBException e) { + // TODO: this needs to be handled more accurately + throw new KafkaException("Error while executing flush from store " + this.topic, e); + } + } + + @Override + public void close() { + flush(); + db.close(); + } + + private static class RocksDbIterator implements KeyValueIterator { + private final RocksIterator iter; + + public RocksDbIterator(RocksIterator iter) { + this.iter = iter; + } + + protected byte[] peekKey() { + return this.getEntry().key(); + } + + protected Entry getEntry() { + return new Entry<>(iter.key(), iter.value()); + } + + @Override + public boolean hasNext() { + return iter.isValid(); + } + + @Override + public Entry next() { + if (!hasNext()) + throw new NoSuchElementException(); + + Entry entry = this.getEntry(); + iter.next(); + + return entry; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("RocksDB iterator does not support remove"); + } + + @Override + public void close() { + } + + } + + private static class LexicographicComparator implements Comparator { + + @Override + public int compare(byte[] left, byte[] right) { + for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) { + int leftByte = left[i] & 0xff; + int rightByte = right[j] & 0xff; + if (leftByte != rightByte) { + return leftByte - rightByte; + } + } + return left.length - right.length; + } + } + + private static class RocksDBRangeIterator extends RocksDbIterator { + // RocksDB's JNI interface does not expose getters/setters that allow the + // comparator to be pluggable, and the default is lexicographic, so it's + // safe to just force lexicographic comparator here for now. + private final Comparator comparator = new LexicographicComparator(); + byte[] to; + + public RocksDBRangeIterator(RocksIterator iter, byte[] from, byte[] to) { + super(iter); + iter.seek(from); + this.to = to; + } + + @Override + public boolean hasNext() { + return super.hasNext() && comparator.compare(super.peekKey(), this.to) < 0; + } + } + + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java new file mode 100644 index 0000000000000..49171e340b356 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -0,0 +1,34 @@ +/** + * 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.kstream; + +import org.apache.kafka.streams.kstream.internals.KStreamImpl; +import org.apache.kafka.streams.processor.TopologyException; +import org.junit.Test; + +public class KStreamBuilderTest { + + @Test(expected = TopologyException.class) + public void testFrom() { + final KStreamBuilder builder = new KStreamBuilder(); + + builder.from("topic-1", "topic-2"); + + builder.addSource(KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.decrementAndGet(), "topic-3"); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java new file mode 100644 index 0000000000000..405c7c93407e6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/FilteredIteratorTest.java @@ -0,0 +1,94 @@ +/** + * 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.kstream.internals; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Iterator; +import java.util.List; + +public class FilteredIteratorTest { + + @Test + public void testFiltering() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = Arrays.asList("3", "9", "6", "3"); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testEmptySource() { + List list = new ArrayList(); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 3 == 0) return i.toString(); + return null; + } + }; + + List expected = new ArrayList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + + @Test + public void testNoMatch() { + List list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5); + + Iterator filtered = new FilteredIterator(list.iterator()) { + protected String filter(Integer i) { + if (i % 7 == 0) return i.toString(); + return null; + } + }; + + List expected = new ArrayList(); + List result = new ArrayList(); + + while (filtered.hasNext()) { + result.add(filtered.next()); + } + + assertEquals(expected, result); + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java new file mode 100644 index 0000000000000..c18ddfe3ca041 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -0,0 +1,90 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +import java.lang.reflect.Array; + +import static org.junit.Assert.assertEquals; + +public class KStreamBranchTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @SuppressWarnings("unchecked") + @Test + public void testKStreamBranch() { + KStreamBuilder builder = new KStreamBuilder(); + + Predicate isEven = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) == 0; + } + }; + Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; + Predicate isOdd = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 2) != 0; + } + }; + + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6}; + + KStream stream; + KStream[] branches; + MockProcessorDef[] processors; + + stream = builder.from(keyDeserializer, valDeserializer, topicName); + branches = stream.branch(isEven, isMultipleOfThree, isOdd); + + assertEquals(3, branches.length); + + processors = (MockProcessorDef[]) Array.newInstance(MockProcessorDef.class, branches.length); + for (int i = 0; i < branches.length; i++) { + processors[i] = new MockProcessorDef<>(); + branches[i].process(processors[i]); + } + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(3, processors[0].processed.size()); + assertEquals(1, processors[1].processed.size()); + assertEquals(2, processors[2].processed.size()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java new file mode 100644 index 0000000000000..b80e1e2028ef7 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -0,0 +1,85 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamFilterTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + private Predicate isMultipleOfThree = new Predicate() { + @Override + public boolean apply(Integer key, String value) { + return (key % 3) == 0; + } + }; + + @Test + public void testFilter() { + KStreamBuilder builder = new KStreamBuilder(); + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStream stream; + MockProcessorDef processor; + + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.filter(isMultipleOfThree).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(2, processor.processed.size()); + } + + @Test + public void testFilterOut() { + KStreamBuilder builder = new KStreamBuilder(); + final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; + + KStream stream; + MockProcessorDef processor; + + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.filterOut(isMultipleOfThree).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(5, processor.processed.size()); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java new file mode 100644 index 0000000000000..e87223e73d886 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -0,0 +1,80 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +import java.util.ArrayList; + +public class KStreamFlatMapTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMap() { + KStreamBuilder builder = new KStreamBuilder(); + + KeyValueMapper>> mapper = + new KeyValueMapper>>() { + @Override + public Iterable> apply(Integer key, String value) { + ArrayList> result = new ArrayList<>(); + for (int i = 0; i < key; i++) { + result.add(KeyValue.pair(Integer.toString(key * 10 + i), value)); + } + return result; + } + }; + + final int[] expectedKeys = {0, 1, 2, 3}; + + KStream stream; + MockProcessorDef processor; + + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.flatMap(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(6, processor.processed.size()); + + String[] expected = {"10:V1", "20:V2", "21:V2", "30:V3", "31:V3", "32:V3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java new file mode 100644 index 0000000000000..09dda65fc3cb6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -0,0 +1,77 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class KStreamFlatMapValuesTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMapValues() { + KStreamBuilder builder = new KStreamBuilder(); + + ValueMapper> mapper = + new ValueMapper>() { + @Override + public Iterable apply(String value) { + ArrayList result = new ArrayList(); + result.add(value.toLowerCase()); + result.add(value); + return result; + } + }; + + final int[] expectedKeys = {0, 1, 2, 3}; + + KStream stream; + MockProcessorDef processor; + + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.flatMapValues(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(8, processor.processed.size()); + + String[] expected = {"0:v0", "0:V0", "1:v1", "1:V1", "2:v2", "2:V2", "3:v3", "3:V3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java new file mode 100644 index 0000000000000..0660ddd87a54f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -0,0 +1,138 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Predicate; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.MockProcessorDef; +import org.apache.kafka.test.UnlimitedWindowDef; +import org.junit.Test; + +import java.util.Collections; + +import static org.junit.Assert.assertEquals; + + +public class KStreamImplTest { + + @Test + public void testNumProcesses() { + final Deserializer deserializer = new StringDeserializer(); + final KStreamBuilder builder = new KStreamBuilder(); + + KStream source1 = builder.from(deserializer, deserializer, "topic-1", "topic-2"); + + KStream source2 = builder.from(deserializer, deserializer, "topic-3", "topic-4"); + + KStream stream1 = + source1.filter(new Predicate() { + @Override + public boolean apply(String key, String value) { + return true; + } + }).filterOut(new Predicate() { + @Override + public boolean apply(String key, String value) { + return false; + } + }); + + KStream stream2 = stream1.mapValues(new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); + + KStream stream3 = source2.flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Collections.singletonList(new Integer(value)); + } + }); + + KStream[] streams2 = stream2.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); + + KStream[] streams3 = stream3.branch( + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return (value % 2) == 0; + } + }, + new Predicate() { + @Override + public boolean apply(String key, Integer value) { + return true; + } + } + ); + + KStream stream4 = streams2[0].with(new UnlimitedWindowDef("window")) + .join(streams3[0].with(new UnlimitedWindowDef("window")), new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }); + + KStream stream5 = streams2[1].with(new UnlimitedWindowDef("window")) + .join(streams3[1].with(new UnlimitedWindowDef("window")), new ValueJoiner() { + @Override + public Integer apply(Integer value1, Integer value2) { + return value1 + value2; + } + }); + + stream4.to("topic-5"); + + stream5.through("topic-6").process(new MockProcessorDef<>()).to("topic-7"); + + assertEquals(2 + // sources + 2 + // stream1 + 1 + // stream2 + 1 + // stream3 + 1 + 2 + // streams2 + 1 + 2 + // streams3 + 2 + 3 + // stream4 + 2 + 3 + // stream5 + 1 + // to + 2 + // through + 1 + // process + 1, // to + builder.build().processors().size()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java new file mode 100644 index 0000000000000..7dea8e08863d2 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamJoinTest.java @@ -0,0 +1,164 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KStreamWindowed; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.ValueJoiner; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; +import org.apache.kafka.test.UnlimitedWindowDef; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamJoinTest { + + private String topic1 = "topic1"; + private String topic2 = "topic2"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + private ValueJoiner joiner = new ValueJoiner() { + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + private ValueMapper valueMapper = new ValueMapper() { + @Override + public String apply(String value) { + return "#" + value; + } + }; + + private ValueMapper> valueMapper2 = new ValueMapper>() { + @Override + public Iterable apply(String value) { + return (Iterable) Utils.mkSet(value); + } + }; + + private KeyValueMapper> keyValueMapper = + new KeyValueMapper>() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(key, value); + } + }; + + KeyValueMapper>> keyValueMapper2 = + new KeyValueMapper>>() { + @Override + public KeyValue> apply(Integer key, String value) { + return KeyValue.pair(key, (Iterable) Utils.mkSet(value)); + } + }; + + + @Test + public void testJoin() { + KStreamBuilder builder = new KStreamBuilder(); + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream1; + KStream stream2; + KStreamWindowed windowed1; + KStreamWindowed windowed2; + MockProcessorDef processor; + String[] expected; + + processor = new MockProcessorDef<>(); + stream1 = builder.from(keyDeserializer, valDeserializer, topic1); + stream2 = builder.from(keyDeserializer, valDeserializer, topic2); + windowed1 = stream1.with(new UnlimitedWindowDef("window1")); + windowed2 = stream2.with(new UnlimitedWindowDef("window2")); + + windowed1.join(windowed2, joiner).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + driver.setTime(0L); + + // push two items to the main stream. the other stream's window is empty + + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } + + assertEquals(0, processor.processed.size()); + + // push two items to the other stream. the main stream's window has two items + + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(2, processor.processed.size()); + + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + + processor.processed.clear(); + + // push all items to the main stream. this should produce two items. + + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } + + assertEquals(2, processor.processed.size()); + + expected = new String[]{"0:X0+Y0", "1:X1+Y1"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + + processor.processed.clear(); + + // there will be previous two items + all items in the main stream's window, thus two are duplicates. + + // push all items to the other stream. this should produce 6 items + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(6, processor.processed.size()); + + expected = new String[]{"0:X0+Y0", "0:X0+Y0", "1:X1+Y1", "1:X1+Y1", "2:X2+Y2", "3:X3+Y3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + + // TODO: test for joinability +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java new file mode 100644 index 0000000000000..bec524f37397a --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -0,0 +1,73 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testMap() { + KStreamBuilder builder = new KStreamBuilder(); + + KeyValueMapper> mapper = + new KeyValueMapper>() { + @Override + public KeyValue apply(Integer key, String value) { + return KeyValue.pair(value, key); + } + }; + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + MockProcessorDef processor; + + processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.map(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = new String[]{"V0:0", "V1:1", "V2:2", "V3:3"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java new file mode 100644 index 0000000000000..b6507fe2f4dd6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -0,0 +1,71 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class KStreamMapValuesTest { + + private String topicName = "topic"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testFlatMapValues() { + KStreamBuilder builder = new KStreamBuilder(); + + ValueMapper mapper = + new ValueMapper() { + @Override + public Integer apply(String value) { + return value.length(); + } + }; + + final int[] expectedKeys = {1, 10, 100, 1000}; + + KStream stream; + MockProcessorDef processor = new MockProcessorDef<>(); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.mapValues(mapper).process(processor); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], Integer.toString(expectedKeys[i])); + } + + assertEquals(4, processor.processed.size()); + + String[] expected = {"1:1", "10:2", "100:3", "1000:4"}; + + for (int i = 0; i < expected.length; i++) { + assertEquals(expected[i], processor.processed.get(i)); + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java new file mode 100644 index 0000000000000..48a9fc3d279e1 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowedTest.java @@ -0,0 +1,91 @@ +/** + * 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.kstream.internals; + +import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.UnlimitedWindowDef; +import org.junit.Test; + +import java.util.Iterator; + +import static org.junit.Assert.assertEquals; + +public class KStreamWindowedTest { + + private String topicName = "topic"; + private String windowName = "MyWindow"; + + private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); + private StringDeserializer valDeserializer = new StringDeserializer(); + + @Test + public void testWindowedStream() { + KStreamBuilder builder = new KStreamBuilder(); + + final int[] expectedKeys = new int[]{0, 1, 2, 3}; + + KStream stream; + WindowDef windowDef; + + windowDef = new UnlimitedWindowDef<>(windowName); + stream = builder.from(keyDeserializer, valDeserializer, topicName); + stream.with(windowDef); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + Window window = (Window) driver.getStateStore(windowName); + driver.setTime(0L); + + // two items in the window + + for (int i = 0; i < 2; i++) { + driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); + } + + assertEquals(1, countItem(window.find(0, 0L))); + assertEquals(1, countItem(window.find(1, 0L))); + assertEquals(0, countItem(window.find(2, 0L))); + assertEquals(0, countItem(window.find(3, 0L))); + + // previous two items + all items, thus two are duplicates, in the window + + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topicName, expectedKeys[i], "Y" + expectedKeys[i]); + } + + assertEquals(2, countItem(window.find(0, 0L))); + assertEquals(2, countItem(window.find(1, 0L))); + assertEquals(1, countItem(window.find(2, 0L))); + assertEquals(1, countItem(window.find(3, 0L))); + } + + + private int countItem(Iterator iter) { + int i = 0; + while (iter.hasNext()) { + i++; + iter.next(); + } + return i; + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java new file mode 100644 index 0000000000000..57a78ff87a896 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -0,0 +1,99 @@ +/** + * 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.processor; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.test.MockProcessorDef; +import org.junit.Test; + +public class TopologyBuilderTest { + + @Test(expected = TopologyException.class) + public void testAddSourceWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSource("source", "topic-2"); + } + + @Test(expected = TopologyException.class) + public void testAddSourceWithSameTopic() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSource("source-2", "topic-1"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addProcessor("processor", new MockProcessorDef(), "source"); + builder.addProcessor("processor", new MockProcessorDef(), "source"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithWrongParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addProcessor("processor", new MockProcessorDef(), "source"); + } + + @Test(expected = TopologyException.class) + public void testAddProcessorWithSelfParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addProcessor("processor", new MockProcessorDef(), "processor"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithSameName() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source", "topic-1"); + builder.addSink("sink", "topic-2", "source"); + builder.addSink("sink", "topic-3", "source"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithWrongParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSink("sink", "topic-2", "source"); + } + + @Test(expected = TopologyException.class) + public void testAddSinkWithSelfParent() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSink("sink", "topic-2", "sink"); + } + + @Test + public void testSourceTopics() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source-1", "topic-1"); + builder.addSource("source-2", "topic-2"); + builder.addSource("source-3", "topic-3"); + + assertEquals(builder.sourceTopics().size(), 3); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java new file mode 100644 index 0000000000000..0a1f95c29e0a8 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/MinTimestampTrackerTest.java @@ -0,0 +1,93 @@ +/** + * 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.processor.internals; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class MinTimestampTrackerTest { + + private Stamped elem(long timestamp) { + return new Stamped<>("", timestamp); + } + + @SuppressWarnings("unchecked") + @Test + public void testTracking() { + TimestampTracker tracker = new MinTimestampTracker<>(); + + Object[] elems = new Object[]{ + elem(100), elem(101), elem(102), elem(98), elem(99), elem(100) + }; + + int insertionIndex = 0; + int removalIndex = 0; + + // add 100 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // add 101 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(101L, tracker.get()); + + // add 102 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(101L, tracker.get()); + + // add 98 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 99 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // add 100 + tracker.addElement((Stamped) elems[insertionIndex++]); + assertEquals(98L, tracker.get()); + + // remove 101 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 102 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(98L, tracker.get()); + + // remove 98 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(99L, tracker.get()); + + // remove 99 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); + + // remove 100 + tracker.removeElement((Stamped) elems[removalIndex++]); + assertEquals(100L, tracker.get()); + + assertEquals(insertionIndex, removalIndex); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java new file mode 100644 index 0000000000000..b91acdce13bfc --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java @@ -0,0 +1,102 @@ +/** + * 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.processor.internals; + +import static org.junit.Assert.assertEquals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +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.Serializer; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.Test; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; + +public class PartitionGroupTest { + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); + private final TopicPartition partition1 = new TopicPartition("topic", 1); + private final TopicPartition partition2 = new TopicPartition("topic", 2); + private final RecordQueue queue1 = new RecordQueue(partition1, new MockSourceNode<>(intDeserializer, intDeserializer)); + private final RecordQueue queue2 = new RecordQueue(partition2, new MockSourceNode<>(intDeserializer, intDeserializer)); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + private final PartitionGroup group = new PartitionGroup(new HashMap() { + { + put(partition1, queue1); + put(partition2, queue2); + } + }, timestampExtractor); + + @Test + public void testTimeTracking() { + assertEquals(0, group.numBuffered()); + + // add three 3 records with timestamp 1, 3, 5 to partition-1 + List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue)); + + group.addRawRecords(partition1, list1); + + // add three 3 records with timestamp 2, 4, 6 to partition-2 + List> list2 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue)); + + group.addRawRecords(partition2, list2); + + assertEquals(6, group.numBuffered()); + assertEquals(3, group.numBuffered(partition1)); + assertEquals(3, group.numBuffered(partition2)); + assertEquals(TimestampTracker.NOT_KNOWN, group.timestamp()); + + StampedRecord record; + PartitionGroup.RecordInfo info = new PartitionGroup.RecordInfo(); + + // get one record + record = group.nextRecord(info); + assertEquals(partition1, info.partition()); + assertEquals(1L, record.timestamp); + assertEquals(5, group.numBuffered()); + assertEquals(2, group.numBuffered(partition1)); + assertEquals(3, group.numBuffered(partition2)); + assertEquals(TimestampTracker.NOT_KNOWN, group.timestamp()); + + // get one record, now the time should be advanced + record = group.nextRecord(info); + assertEquals(partition2, info.partition()); + assertEquals(2L, record.timestamp); + assertEquals(4, group.numBuffered()); + assertEquals(2, group.numBuffered(partition1)); + assertEquals(2, group.numBuffered(partition2)); + assertEquals(3L, group.timestamp()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java new file mode 100644 index 0000000000000..343ed52b7ef54 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -0,0 +1,449 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.common.Node; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +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.Serializer; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.processor.RestoreFunc; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.state.OffsetCheckpoint; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.channels.FileLock; +import java.nio.file.Files; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +public class ProcessorStateManagerTest { + + private static class MockStateStore implements StateStore { + private final String name; + private final boolean persistent; + + public boolean flushed = false; + public boolean closed = false; + public final ArrayList keys = new ArrayList<>(); + + public MockStateStore(String name, boolean persistent) { + this.name = name; + this.persistent = persistent; + } + @Override + public String name() { + return name; + } + @Override + public void flush() { + flushed = true; + } + @Override + public void close() { + closed = true; + } + @Override + public boolean persistent() { + return persistent; + } + + public final RestoreFunc restoreFunc = new RestoreFunc() { + private final Deserializer deserializer = new IntegerDeserializer(); + + @Override + public void apply(byte[] key, byte[] value) { + keys.add(deserializer.deserialize("", key)); + } + }; + } + + private class MockRestoreConsumer extends MockConsumer { + private final Serializer serializer = new IntegerSerializer(); + + public TopicPartition assignedPartition = null; + public TopicPartition seekPartition = null; + public long seekOffset = -1L; + public boolean seekToBeginingCalled = false; + public boolean seekToEndCalled = false; + private long endOffset = 0L; + private long currentOffset = 0L; + + private ArrayList> recordBuffer = new ArrayList<>(); + + MockRestoreConsumer() { + super(OffsetResetStrategy.EARLIEST); + + reset(); + } + + // reset this mock restore consumer for a state store registration + public void reset() { + assignedPartition = null; + seekOffset = -1L; + seekToBeginingCalled = false; + seekToEndCalled = false; + endOffset = 0L; + recordBuffer.clear(); + } + + // buffer a record (we cannot use addRecord because we need to add records before asigning a partition) + public void bufferRecord(ConsumerRecord record) { + recordBuffer.add( + new ConsumerRecord<>(record.topic(), record.partition(), record.offset(), + serializer.serialize(record.topic(), record.key()), + serializer.serialize(record.topic(), record.value()))); + endOffset = record.offset(); + + super.updateEndOffsets(Collections.singletonMap(assignedPartition, endOffset)); + } + + @Override + public synchronized void assign(List partitions) { + int numPartitions = partitions.size(); + if (numPartitions > 1) + throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); + + if (numPartitions == 1) { + if (assignedPartition != null) + throw new IllegalStateException("RestoreConsumer: partition already assigned"); + assignedPartition = partitions.get(0); + + // set the beginning offset to 0 + // NOTE: this is users responsible to set the initial lEO. + super.updateBeginningOffsets(Collections.singletonMap(assignedPartition, 0L)); + } + + super.assign(partitions); + } + + @Override + public ConsumerRecords poll(long timeout) { + // add buffered records to MockConsumer + for (ConsumerRecord record : recordBuffer) { + super.addRecord(record); + } + recordBuffer.clear(); + + ConsumerRecords records = super.poll(timeout); + + // set the current offset + Iterable> partitionRecords = records.records(assignedPartition); + for (ConsumerRecord record : partitionRecords) { + currentOffset = record.offset(); + } + + return records; + } + + @Override + public synchronized long position(TopicPartition partition) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: unassigned partition"); + + return currentOffset; + } + + @Override + public synchronized void seek(TopicPartition partition, long offset) { + if (offset < 0) + throw new IllegalArgumentException("RestoreConsumer: offset should not be negative"); + + if (seekOffset >= 0) + throw new IllegalStateException("RestoreConsumer: offset already seeked"); + + seekPartition = partition; + seekOffset = offset; + currentOffset = offset; + super.seek(partition, offset); + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + if (partitions.length != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + seekToBeginingCalled = true; + currentOffset = 0L; + } + + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + if (partitions.length != 1) + throw new IllegalStateException("RestoreConsumer: other than one partition specified"); + + for (TopicPartition partition : partitions) { + if (!partition.equals(assignedPartition)) + throw new IllegalStateException("RestoreConsumer: seek-to-end not on the assigned partition"); + } + + seekToEndCalled = true; + currentOffset = endOffset; + } + } + + @Test + public void testLockStateDirectory() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + FileLock lock; + + // the state manager locks the directory + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, new MockRestoreConsumer()); + + try { + // this should not get the lock + lock = ProcessorStateManager.lockStateDirectory(baseDir); + assertNull(lock); + } finally { + // by closing the state manager, release the lock + stateMgr.close(Collections.emptyMap()); + } + + // now, this should get the lock + lock = ProcessorStateManager.lockStateDirectory(baseDir); + try { + assertNotNull(lock); + } finally { + if (lock != null) lock.release(); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test(expected = IllegalStateException.class) + public void testNoTopic() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + MockStateStore mockStateStore = new MockStateStore("mockStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, new MockRestoreConsumer()); + try { + stateMgr.register(mockStateStore, mockStateStore.restoreFunc); + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testRegisterPersistentStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + long lastCheckpointedOffset = 10L; + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); + checkpoint.write(Collections.singletonMap(new TopicPartition("persistentStore", 2), lastCheckpointedOffset)); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("persistentStore", Arrays.asList( + new PartitionInfo("persistentStore", 1, Node.noNode(), new Node[0], new Node[0]), + new PartitionInfo("persistentStore", 2, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updateEndOffsets(Collections.singletonMap(new TopicPartition("persistentStore", 2), 13L)); + + MockStateStore persistentStore = new MockStateStore("persistentStore", false); // non persistent store + + ProcessorStateManager stateMgr = new ProcessorStateManager(2, baseDir, restoreConsumer); + try { + restoreConsumer.reset(); + + ArrayList expectedKeys = new ArrayList<>(); + for (int i = 1; i <= 3; i++) { + long offset = (long) i; + int key = i * 10; + expectedKeys.add(key); + restoreConsumer.bufferRecord( + new ConsumerRecord<>("persistentStore", 2, offset, key, 0) + ); + } + + stateMgr.register(persistentStore, persistentStore.restoreFunc); + + assertEquals(new TopicPartition("persistentStore", 2), restoreConsumer.assignedPartition); + assertEquals(lastCheckpointedOffset, restoreConsumer.seekOffset); + assertFalse(restoreConsumer.seekToBeginingCalled); + assertTrue(restoreConsumer.seekToEndCalled); + assertEquals(expectedKeys, persistentStore.keys); + + } finally { + stateMgr.close(Collections.emptyMap()); + } + + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testRegisterNonPersistentStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + long lastCheckpointedOffset = 10L; + OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME)); + checkpoint.write(Collections.singletonMap(new TopicPartition("persistentStore", 2), lastCheckpointedOffset)); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("nonPersistentStore", Arrays.asList( + new PartitionInfo("nonPersistentStore", 1, Node.noNode(), new Node[0], new Node[0]), + new PartitionInfo("nonPersistentStore", 2, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updateEndOffsets(Collections.singletonMap(new TopicPartition("persistentStore", 2), 13L)); + + MockStateStore nonPersistentStore = new MockStateStore("nonPersistentStore", true); // persistent store + + ProcessorStateManager stateMgr = new ProcessorStateManager(2, baseDir, restoreConsumer); + try { + restoreConsumer.reset(); + + ArrayList expectedKeys = new ArrayList<>(); + for (int i = 1; i <= 3; i++) { + long offset = (long) (i + 100); + int key = i; + expectedKeys.add(i); + restoreConsumer.bufferRecord( + new ConsumerRecord<>("nonPersistentStore", 2, offset, key, 0) + ); + } + + stateMgr.register(nonPersistentStore, nonPersistentStore.restoreFunc); + + assertEquals(new TopicPartition("nonPersistentStore", 2), restoreConsumer.assignedPartition); + assertEquals(0L, restoreConsumer.seekOffset); + assertTrue(restoreConsumer.seekToBeginingCalled); + assertTrue(restoreConsumer.seekToEndCalled); + assertEquals(expectedKeys, nonPersistentStore.keys); + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testGetStore() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("mockStore", Arrays.asList( + new PartitionInfo("mockStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + + MockStateStore mockStateStore = new MockStateStore("mockStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, restoreConsumer); + try { + stateMgr.register(mockStateStore, mockStateStore.restoreFunc); + + assertNull(stateMgr.getStore("noSuchStore")); + assertEquals(mockStateStore, stateMgr.getStore("mockStore")); + + } finally { + stateMgr.close(Collections.emptyMap()); + } + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testClose() throws IOException { + File baseDir = Files.createTempDirectory("test").toFile(); + File checkpointFile = new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME); + try { + // write an empty checkpoint file + OffsetCheckpoint oldCheckpoint = new OffsetCheckpoint(checkpointFile); + oldCheckpoint.write(Collections.emptyMap()); + + MockRestoreConsumer restoreConsumer = new MockRestoreConsumer(); + restoreConsumer.updatePartitions("persistentStore", Arrays.asList( + new PartitionInfo("persistentStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + restoreConsumer.updatePartitions("nonPersistentStore", Arrays.asList( + new PartitionInfo("nonPersistentStore", 1, Node.noNode(), new Node[0], new Node[0]) + )); + + // set up ack'ed offsets + HashMap ackedOffsets = new HashMap<>(); + ackedOffsets.put(new TopicPartition("persistentStore", 1), 123L); + ackedOffsets.put(new TopicPartition("nonPersistentStore", 1), 456L); + ackedOffsets.put(new TopicPartition("otherTopic", 1), 789L); + + MockStateStore persistentStore = new MockStateStore("persistentStore", true); + MockStateStore nonPersistentStore = new MockStateStore("nonPersistentStore", false); + + ProcessorStateManager stateMgr = new ProcessorStateManager(1, baseDir, restoreConsumer); + try { + // make sure the checkpoint file is deleted + assertFalse(checkpointFile.exists()); + + restoreConsumer.reset(); + stateMgr.register(persistentStore, persistentStore.restoreFunc); + + restoreConsumer.reset(); + stateMgr.register(nonPersistentStore, nonPersistentStore.restoreFunc); + } finally { + // close the state manager with the ack'ed offsets + stateMgr.close(ackedOffsets); + } + + // make sure all stores are closed, and the checkpoint file is written. + assertTrue(persistentStore.flushed); + assertTrue(persistentStore.closed); + assertTrue(nonPersistentStore.flushed); + assertTrue(nonPersistentStore.closed); + assertTrue(checkpointFile.exists()); + + // the checkpoint file should contain an offset from the persistent store only. + OffsetCheckpoint newCheckpoint = new OffsetCheckpoint(checkpointFile); + Map checkpointedOffsets = newCheckpoint.read(); + assertEquals(1, checkpointedOffsets.size()); + assertEquals(new Long(123L + 1L), checkpointedOffsets.get(new TopicPartition("persistentStore", 1))); + } finally { + Utils.delete(baseDir); + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java new file mode 100644 index 0000000000000..1abb989e30de2 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -0,0 +1,326 @@ +/** + * 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.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorDef; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.state.InMemoryKeyValueStore; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.test.MockProcessorDef; +import org.apache.kafka.test.ProcessorTopologyTestDriver; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.FileVisitResult; +import java.nio.file.Files; +import java.nio.file.Path; +import java.nio.file.SimpleFileVisitor; +import java.nio.file.attribute.BasicFileAttributes; +import java.util.Properties; + +public class ProcessorTopologyTest { + + private static final Serializer STRING_SERIALIZER = new StringSerializer(); + private static final Deserializer STRING_DESERIALIZER = new StringDeserializer(); + private static final File STATE_DIR = new File("build/data").getAbsoluteFile(); + + protected static final String INPUT_TOPIC = "input-topic"; + protected static final String OUTPUT_TOPIC_1 = "output-topic-1"; + protected static final String OUTPUT_TOPIC_2 = "output-topic-2"; + + private static long timestamp = 1000L; + + private ProcessorTopologyTestDriver driver; + private StreamingConfig config; + + @Before + public void setup() { + STATE_DIR.mkdirs(); + Properties props = new Properties(); + props.setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, STATE_DIR.getAbsolutePath()); + props.setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName()); + props.setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); + props.setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); + this.config = new StreamingConfig(props); + } + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + if (STATE_DIR.exists()) { + try { + Files.walkFileTree(STATE_DIR.toPath(), new SimpleFileVisitor() { + @Override + public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException { + Files.delete(file); + return FileVisitResult.CONTINUE; + } + + @Override + public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException { + Files.delete(dir); + return FileVisitResult.CONTINUE; + } + + }); + } catch (IOException e) { + // do nothing + } + } + } + + @Test + public void testTopologyMetadata() { + final TopologyBuilder builder = new TopologyBuilder(); + + builder.addSource("source-1", "topic-1"); + builder.addSource("source-2", "topic-2", "topic-3"); + builder.addProcessor("processor-1", new MockProcessorDef(), "source-1"); + builder.addProcessor("processor-2", new MockProcessorDef(), "source-1", "source-2"); + builder.addSink("sink-1", "topic-3", "processor-1"); + builder.addSink("sink-2", "topic-4", "processor-1", "processor-2"); + + final ProcessorTopology topology = builder.build(); + + assertEquals(6, topology.processors().size()); + + assertEquals(2, topology.sources().size()); + + assertEquals(3, topology.sourceTopics().size()); + + assertNotNull(topology.source("topic-1")); + + assertNotNull(topology.source("topic-2")); + + assertNotNull(topology.source("topic-3")); + + assertEquals(topology.source("topic-2"), topology.source("topic-3")); + } + + @Test + public void testDrivingSimpleTopology() { + driver = new ProcessorTopologyTestDriver(config, createSimpleTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2"); + assertNoOutputRecord(OUTPUT_TOPIC_2); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_2); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5"); + } + + @Test + public void testDrivingMultiplexingTopology() { + driver = new ProcessorTopologyTestDriver(config, createMultiplexingTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1(2)"); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2(2)"); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key4", "value4(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key5", "value5(2)"); + } + + @Test + public void testDrivingStatefulTopology() { + String storeName = "entries"; + driver = new ProcessorTopologyTestDriver(config, createStatefulTopology(storeName), storeName); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key1", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + assertNoOutputRecord(OUTPUT_TOPIC_1); + + KeyValueStore store = driver.getKeyValueStore("entries"); + assertEquals("value4", store.get("key1")); + assertEquals("value2", store.get("key2")); + assertEquals("value3", store.get("key3")); + assertNull(store.get("key4")); + } + + protected void assertNextOutputRecord(String topic, String key, String value) { + assertProducerRecord(driver.readOutput(topic, STRING_DESERIALIZER, STRING_DESERIALIZER), topic, key, value); + } + + protected void assertNoOutputRecord(String topic) { + assertNull(driver.readOutput(topic)); + } + + private void assertProducerRecord(ProducerRecord record, String topic, String key, String value) { + assertEquals(topic, record.topic()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + // Kafka Streaming doesn't set the partition, so it's always null + assertNull(record.partition()); + } + + protected TopologyBuilder createSimpleTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new ForwardingProcessor()), "source") + .addSink("sink", OUTPUT_TOPIC_1, "processor"); + } + + protected TopologyBuilder createMultiplexingTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new MultiplexingProcessor(2)), "source") + .addSink("sink1", OUTPUT_TOPIC_1, "processor") + .addSink("sink2", OUTPUT_TOPIC_2, "processor"); + } + + protected TopologyBuilder createStatefulTopology(String storeName) { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new StatefulProcessor(storeName)), "source") + .addSink("counts", OUTPUT_TOPIC_1, "processor"); + } + + /** + * A processor that simply forwards all messages to all children. + */ + protected static class ForwardingProcessor extends AbstractProcessor { + + @Override + public void process(String key, String value) { + context().forward(key, value); + } + + @Override + public void punctuate(long streamTime) { + context().forward(Long.toString(streamTime), "punctuate"); + } + } + + /** + * A processor that forwards slightly-modified messages to each child. + */ + protected static class MultiplexingProcessor extends AbstractProcessor { + + private final int numChildren; + + public MultiplexingProcessor(int numChildren) { + this.numChildren = numChildren; + } + + @Override + public void process(String key, String value) { + for (int i = 0; i != numChildren; ++i) { + context().forward(key, value + "(" + (i + 1) + ")", i); + } + } + + @Override + public void punctuate(long streamTime) { + for (int i = 0; i != numChildren; ++i) { + context().forward(Long.toString(streamTime), "punctuate(" + (i + 1) + ")", i); + } + } + } + + /** + * A processor that stores each key-value pair in an in-memory key-value store registered with the context. When + * {@link #punctuate(long)} is called, it outputs the total number of entries in the store. + */ + protected static class StatefulProcessor extends AbstractProcessor { + + private KeyValueStore store; + private final String storeName; + + public StatefulProcessor(String storeName) { + this.storeName = storeName; + } + + @Override + public void init(ProcessorContext context) { + super.init(context); + store = new InMemoryKeyValueStore<>(storeName, context); + } + + @Override + public void process(String key, String value) { + store.put(key, value); + } + + @Override + public void punctuate(long streamTime) { + int count = 0; + for (KeyValueIterator iter = store.all(); iter.hasNext();) { + iter.next(); + ++count; + } + context().forward(Long.toString(streamTime), count); + } + } + + protected ProcessorDef define(final Processor processor) { + return new ProcessorDef() { + @Override + public Processor instance() { + return processor; + } + }; + } + + public static class CustomTimestampExtractor implements TimestampExtractor { + @Override + public long extract(ConsumerRecord record) { + return timestamp; + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java new file mode 100644 index 0000000000000..b1403bd02f44c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PunctuationQueueTest.java @@ -0,0 +1,85 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.junit.Test; + +import java.util.ArrayList; + +import static org.junit.Assert.assertEquals; + +public class PunctuationQueueTest { + + @Test + public void testPunctuationInterval() { + TestProcessor processor = new TestProcessor(); + ProcessorNode node = new ProcessorNode<>("test", processor); + PunctuationQueue queue = new PunctuationQueue(); + + PunctuationSchedule sched = new PunctuationSchedule(node, 100L); + final long now = sched.timestamp - 100L; + + queue.schedule(sched); + + Punctuator punctuator = new Punctuator() { + public void punctuate(ProcessorNode node, long time) { + node.processor().punctuate(time); + } + }; + + queue.mayPunctuate(now, punctuator); + assertEquals(0, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 99L, punctuator); + assertEquals(0, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 100L, punctuator); + assertEquals(1, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 199L, punctuator); + assertEquals(1, processor.punctuatedAt.size()); + + queue.mayPunctuate(now + 200L, punctuator); + assertEquals(2, processor.punctuatedAt.size()); + } + + private static class TestProcessor implements Processor { + + public final ArrayList punctuatedAt = new ArrayList<>(); + + @Override + public void init(ProcessorContext context) { + } + + @Override + public void process(String key, String value) { + } + + @Override + public void punctuate(long streamTime) { + punctuatedAt.add(streamTime); + } + + @Override + public void close() { + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java new file mode 100644 index 0000000000000..6e86410b19ed6 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordQueueTest.java @@ -0,0 +1,116 @@ +/** + * 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.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +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.Serializer; +import org.apache.kafka.streams.processor.TimestampExtractor; +import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; +import org.junit.Test; + +import java.util.Arrays; +import java.util.List; + +public class RecordQueueTest { + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final TimestampExtractor timestampExtractor = new MockTimestampExtractor(); + private final RecordQueue queue = new RecordQueue(new TopicPartition("topic", 1), new MockSourceNode<>(intDeserializer, intDeserializer)); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + @Test + public void testTimeTracking() { + + assertTrue(queue.isEmpty()); + + // add three 3 out-of-order records with timestamp 2, 1, 3 + List> list1 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3, recordKey, recordValue)); + + queue.addRawRecords(list1, timestampExtractor); + + assertEquals(3, queue.size()); + assertEquals(TimestampTracker.NOT_KNOWN, queue.timestamp()); + + // poll the first record, now with 1, 3 + assertEquals(2L, queue.poll().timestamp); + assertEquals(2, queue.size()); + assertEquals(1L, queue.timestamp()); + + // poll the second record, now with 3 + assertEquals(1L, queue.poll().timestamp); + assertEquals(1, queue.size()); + assertEquals(3L, queue.timestamp()); + + // add three 3 out-of-order records with timestamp 4, 1, 2 + // now with 3, 4, 1, 2 + List> list2 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 1, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 2, recordKey, recordValue)); + + queue.addRawRecords(list2, timestampExtractor); + + assertEquals(4, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll the third record, now with 4, 1, 2 + assertEquals(3L, queue.poll().timestamp); + assertEquals(3, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll the rest records + assertEquals(4L, queue.poll().timestamp); + assertEquals(3L, queue.timestamp()); + + assertEquals(1L, queue.poll().timestamp); + assertEquals(3L, queue.timestamp()); + + assertEquals(2L, queue.poll().timestamp); + assertEquals(0, queue.size()); + assertEquals(3L, queue.timestamp()); + + // add three more records with 4, 5, 6 + List> list3 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 4, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 6, recordKey, recordValue)); + + queue.addRawRecords(list3, timestampExtractor); + + assertEquals(3, queue.size()); + assertEquals(3L, queue.timestamp()); + + // poll one record again, the timestamp should advance now + assertEquals(4L, queue.poll().timestamp); + assertEquals(2, queue.size()); + assertEquals(5L, queue.timestamp()); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java new file mode 100644 index 0000000000000..8dcfc404f66d7 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -0,0 +1,186 @@ +/** + * 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.processor.internals; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +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.Serializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.test.MockSourceNode; +import org.junit.Test; +import org.junit.Before; + +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Properties; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class StreamTaskTest { + + private final Serializer intSerializer = new IntegerSerializer(); + private final Deserializer intDeserializer = new IntegerDeserializer(); + private final Serializer bytesSerializer = new ByteArraySerializer(); + + private final TopicPartition partition1 = new TopicPartition("topic1", 1); + private final TopicPartition partition2 = new TopicPartition("topic2", 1); + private final HashSet partitions = new HashSet<>(Arrays.asList(partition1, partition2)); + + private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final ProcessorTopology topology = new ProcessorTopology( + Arrays.asList((ProcessorNode) source1, (ProcessorNode) source2), + new HashMap() { + { + put("topic1", source1); + put("topic2", source2); + } + }); + + private final StreamingConfig config = new StreamingConfig(new Properties() { + { + setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); + setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); + setProperty(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + } + }); + + private final MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + private final MockProducer producer = new MockProducer<>(false, bytesSerializer, bytesSerializer); + private final MockConsumer restoreStateConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + + private final byte[] recordValue = intSerializer.serialize(null, 10); + private final byte[] recordKey = intSerializer.serialize(null, 1); + + + @Before + public void setup() { + consumer.assign(Arrays.asList(partition1, partition2)); + } + + @SuppressWarnings("unchecked") + @Test + public void testProcessOrder() { + StreamTask task = new StreamTask(0, consumer, producer, restoreStateConsumer, partitions, topology, config); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, recordKey, recordValue) + )); + + task.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, recordKey, recordValue) + )); + + assertEquals(task.process(), 5); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 0); + + assertEquals(task.process(), 4); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 1); + + assertEquals(task.process(), 3); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 1); + + assertEquals(task.process(), 2); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 1); + + assertEquals(task.process(), 1); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 2); + + assertEquals(task.process(), 0); + assertEquals(source1.numReceived, 3); + assertEquals(source2.numReceived, 3); + + task.close(); + } + + @SuppressWarnings("unchecked") + @Test + public void testPauseResume() { + StreamTask task = new StreamTask(1, consumer, producer, restoreStateConsumer, partitions, topology, config); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, recordKey, recordValue) + )); + + task.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 55, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 65, recordKey, recordValue) + )); + + assertEquals(task.process(), 5); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 0); + + assertEquals(consumer.paused().size(), 1); + assertTrue(consumer.paused().contains(partition2)); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 50, recordKey, recordValue) + )); + + assertEquals(consumer.paused().size(), 2); + assertTrue(consumer.paused().contains(partition1)); + assertTrue(consumer.paused().contains(partition2)); + + assertEquals(task.process(), 7); + assertEquals(source1.numReceived, 1); + assertEquals(source2.numReceived, 1); + + assertEquals(consumer.paused().size(), 1); + assertTrue(consumer.paused().contains(partition1)); + + assertEquals(task.process(), 6); + assertEquals(source1.numReceived, 2); + assertEquals(source2.numReceived, 1); + + assertEquals(consumer.paused().size(), 0); + + task.close(); + } + + private Iterable> records(ConsumerRecord... recs) { + return Arrays.asList(recs); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java new file mode 100644 index 0000000000000..1f3e54135f504 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -0,0 +1,389 @@ +/** + * 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.processor.internals; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.SystemTime; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.junit.Test; + +import java.io.File; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.Set; + +public class StreamThreadTest { + + private TopicPartition t1p1 = new TopicPartition("topic1", 1); + private TopicPartition t1p2 = new TopicPartition("topic1", 2); + private TopicPartition t2p1 = new TopicPartition("topic2", 1); + private TopicPartition t2p2 = new TopicPartition("topic2", 2); + + private Properties configProps() { + return new Properties() { + { + setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); + setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); + setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); + setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); + setProperty(StreamingConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + } + }; + } + + private static class TestStreamTask extends StreamTask { + public boolean committed = false; + + public TestStreamTask(int id, + Consumer consumer, + Producer producer, + Consumer restoreConsumer, + Collection partitions, + ProcessorTopology topology, + StreamingConfig config) { + super(id, consumer, producer, restoreConsumer, partitions, topology, config); + } + + @Override + public void commit() { + super.commit(); + committed = true; + } + } + + private ByteArraySerializer serializer = new ByteArraySerializer(); + + @SuppressWarnings("unchecked") + @Test + public void testPartitionAssignmentChange() throws Exception { + StreamingConfig config = new StreamingConfig(configProps()); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.LATEST); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + builder.addSource("source2", "topic2"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, new SystemTime()) { + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + assertTrue(thread.tasks().isEmpty()); + + List revokedPartitions; + List assignedPartitions; + Set expectedGroup1; + Set expectedGroup2; + + revokedPartitions = Collections.emptyList(); + assignedPartitions = Collections.singletonList(t1p1); + expectedGroup1 = new HashSet<>(Arrays.asList(t1p1)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(1)); + assertEquals(expectedGroup1, thread.tasks().get(1).partitions()); + assertEquals(1, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Collections.singletonList(t1p2); + expectedGroup2 = new HashSet<>(Arrays.asList(t1p2)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(2)); + assertEquals(expectedGroup2, thread.tasks().get(2).partitions()); + assertEquals(1, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Arrays.asList(t1p1, t1p2); + expectedGroup1 = new HashSet<>(Collections.singleton(t1p1)); + expectedGroup2 = new HashSet<>(Collections.singleton(t1p2)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(1)); + assertTrue(thread.tasks().containsKey(2)); + assertEquals(expectedGroup1, thread.tasks().get(1).partitions()); + assertEquals(expectedGroup2, thread.tasks().get(2).partitions()); + assertEquals(2, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Arrays.asList(t1p1, t1p2, t2p1, t2p2); + expectedGroup1 = new HashSet<>(Arrays.asList(t1p1, t2p1)); + expectedGroup2 = new HashSet<>(Arrays.asList(t1p2, t2p2)); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().containsKey(1)); + assertTrue(thread.tasks().containsKey(2)); + assertEquals(expectedGroup1, thread.tasks().get(1).partitions()); + assertEquals(expectedGroup2, thread.tasks().get(2).partitions()); + assertEquals(2, thread.tasks().size()); + + revokedPartitions = assignedPartitions; + assignedPartitions = Collections.emptyList(); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertTrue(thread.tasks().isEmpty()); + } + + @Test + public void testMaybeClean() throws Exception { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + final long cleanupDelay = 1000L; + Properties props = configProps(); + props.setProperty(StreamingConfig.STATE_CLEANUP_DELAY_MS_CONFIG, Long.toString(cleanupDelay)); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + + StreamingConfig config = new StreamingConfig(props); + + File stateDir1 = new File(baseDir, "1"); + File stateDir2 = new File(baseDir, "2"); + File stateDir3 = new File(baseDir, "3"); + File extraDir = new File(baseDir, "X"); + stateDir1.mkdir(); + stateDir2.mkdir(); + stateDir3.mkdir(); + extraDir.mkdir(); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.LATEST); + MockTime mockTime = new MockTime(); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, mockTime) { + @Override + public void maybeClean() { + super.maybeClean(); + } + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + assertTrue(thread.tasks().isEmpty()); + mockTime.sleep(cleanupDelay); + + // all directories exist since an assignment didn't happen + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertTrue(stateDir3.exists()); + assertTrue(extraDir.exists()); + + List revokedPartitions; + List assignedPartitions; + Map prevTasks; + + // + // Assign t1p1 and t1p2. This should create Task 1 & 2 + // + revokedPartitions = Collections.emptyList(); + assignedPartitions = Arrays.asList(t1p1, t1p2); + prevTasks = new HashMap(thread.tasks()); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + // there shouldn't be any previous task + assertTrue(prevTasks.isEmpty()); + + // task 1 & 2 are created + assertEquals(2, thread.tasks().size()); + + // all directories should still exit before the cleanup delay time + mockTime.sleep(cleanupDelay - 10L); + thread.maybeClean(); + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertTrue(stateDir3.exists()); + assertTrue(extraDir.exists()); + + // all state directories except for task 1 & 2 will be removed. the extra directory should still exists + mockTime.sleep(11L); + thread.maybeClean(); + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertFalse(stateDir3.exists()); + assertTrue(extraDir.exists()); + + // + // Revoke t1p1 and t1p2. This should remove Task 1 & 2 + // + revokedPartitions = assignedPartitions; + assignedPartitions = Collections.emptyList(); + prevTasks = new HashMap(thread.tasks()); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + // previous tasks should be committed + assertEquals(2, prevTasks.size()); + for (StreamTask task : prevTasks.values()) { + assertTrue(((TestStreamTask) task).committed); + ((TestStreamTask) task).committed = false; + } + + // no task + assertTrue(thread.tasks().isEmpty()); + + // all state directories for task 1 & 2 still exist before the cleanup delay time + mockTime.sleep(cleanupDelay - 10L); + thread.maybeClean(); + assertTrue(stateDir1.exists()); + assertTrue(stateDir2.exists()); + assertFalse(stateDir3.exists()); + assertTrue(extraDir.exists()); + + // all state directories for task 1 & 2 are removed + mockTime.sleep(11L); + thread.maybeClean(); + assertFalse(stateDir1.exists()); + assertFalse(stateDir2.exists()); + assertFalse(stateDir3.exists()); + assertTrue(extraDir.exists()); + + } finally { + Utils.delete(baseDir); + } + } + + @Test + public void testMaybeCommit() throws Exception { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + final long commitInterval = 1000L; + Properties props = configProps(); + props.setProperty(StreamingConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + props.setProperty(StreamingConfig.COMMIT_INTERVAL_MS_CONFIG, Long.toString(commitInterval)); + + StreamingConfig config = new StreamingConfig(props); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + final MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + MockTime mockTime = new MockTime(); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addSource("source1", "topic1"); + + StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, mockTime) { + @Override + public void maybeCommit() { + super.maybeCommit(); + } + @Override + protected StreamTask createStreamTask(int id, Collection partitionsForTask) { + return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config); + } + }; + + ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener; + + List revokedPartitions; + List assignedPartitions; + + // + // Assign t1p1 and t1p2. This should create Task 1 & 2 + // + revokedPartitions = Collections.emptyList(); + assignedPartitions = Arrays.asList(t1p1, t1p2); + + rebalanceListener.onPartitionsRevoked(revokedPartitions); + rebalanceListener.onPartitionsAssigned(assignedPartitions); + + assertEquals(2, thread.tasks().size()); + + // no task is committed before the commit interval + mockTime.sleep(commitInterval - 10L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertFalse(((TestStreamTask) task).committed); + } + + // all tasks are committed after the commit interval + mockTime.sleep(11L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertTrue(((TestStreamTask) task).committed); + ((TestStreamTask) task).committed = false; + } + + // no task is committed before the commit interval, again + mockTime.sleep(commitInterval - 10L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertFalse(((TestStreamTask) task).committed); + } + + // all tasks are committed after the commit interval, again + mockTime.sleep(11L); + thread.maybeCommit(); + for (StreamTask task : thread.tasks().values()) { + assertTrue(((TestStreamTask) task).committed); + ((TestStreamTask) task).committed = false; + } + + } finally { + Utils.delete(baseDir); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java new file mode 100644 index 0000000000000..2c42e6c225c90 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -0,0 +1,95 @@ +/** + * 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.test; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.internals.ProcessorNode; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; + +import java.util.List; + +public class KStreamTestDriver { + + private final ProcessorTopology topology; + private final MockProcessorContext context; + private ProcessorNode currNode; + + public KStreamTestDriver(KStreamBuilder builder) { + this(builder, null, null); + } + + public KStreamTestDriver(KStreamBuilder builder, Serializer serializer, Deserializer deserializer) { + this.topology = builder.build(); + this.context = new MockProcessorContext(this, serializer, deserializer); + + for (ProcessorNode node : topology.processors()) { + currNode = node; + try { + node.init(context); + } finally { + currNode = null; + } + } + } + + public void process(String topicName, Object key, Object value) { + currNode = topology.source(topicName); + try { + forward(key, value); + } finally { + currNode = null; + } + } + + public void setTime(long timestamp) { + context.setTime(timestamp); + } + + public StateStore getStateStore(String name) { + return context.getStateStore(name); + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + } + + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + ProcessorNode thisNode = currNode; + ProcessorNode childNode = (ProcessorNode) thisNode.children().get(childIndex); + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + } + +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java new file mode 100644 index 0000000000000..3fdfc82c1ff76 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -0,0 +1,143 @@ +/** + * 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.test; + +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.RestoreFunc; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; + +public class MockProcessorContext implements ProcessorContext { + + private final KStreamTestDriver driver; + private final Serializer serializer; + private final Deserializer deserializer; + + private Map storeMap = new HashMap<>(); + + long timestamp = -1L; + + public MockProcessorContext(KStreamTestDriver driver, Serializer serializer, Deserializer deserializer) { + this.driver = driver; + this.serializer = serializer; + this.deserializer = deserializer; + } + + public void setTime(long timestamp) { + this.timestamp = timestamp; + } + + public int id() { + return -1; + } + + @Override + public boolean joinable() { + return true; + } + + @Override + public Serializer keySerializer() { + return serializer; + } + + @Override + public Serializer valueSerializer() { + return serializer; + } + + @Override + public Deserializer keyDeserializer() { + return deserializer; + } + + @Override + public Deserializer valueDeserializer() { + return deserializer; + } + + @Override + public File stateDir() { + throw new UnsupportedOperationException("stateDir() not supported."); + } + + @Override + public Metrics metrics() { + throw new UnsupportedOperationException("metrics() not supported."); + } + + @Override + public void register(StateStore store, RestoreFunc func) { + if (func != null) throw new UnsupportedOperationException("RestoreFunc not supported."); + storeMap.put(store.name(), store); + } + + @Override + public StateStore getStateStore(String name) { + return storeMap.get(name); + } + + @Override + public void schedule(long interval) { + throw new UnsupportedOperationException("schedule() not supported"); + } + + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value) { + driver.forward(key, value); + } + + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value, int childIndex) { + driver.forward(key, value, childIndex); + } + + @Override + public void commit() { + throw new UnsupportedOperationException("commit() not supported."); + } + + @Override + public String topic() { + throw new UnsupportedOperationException("topic() not supported."); + } + + @Override + public int partition() { + throw new UnsupportedOperationException("partition() not supported."); + } + + @Override + public long offset() { + throw new UnsupportedOperationException("offset() not supported."); + } + + @Override + public long timestamp() { + return this.timestamp; + } + +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java new file mode 100644 index 0000000000000..918b4683317f9 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorDef.java @@ -0,0 +1,58 @@ +/** + * 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.test; + +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorDef; + +import java.util.ArrayList; + +public class MockProcessorDef implements ProcessorDef { + + public final ArrayList processed = new ArrayList<>(); + public final ArrayList punctuated = new ArrayList<>(); + + public Processor instance() { + return new MockProcessor(); + } + + public class MockProcessor implements Processor { + + @Override + public void init(ProcessorContext context) { + // do nothing + } + + @Override + public void process(K key, V value) { + processed.add(key + ":" + value); + } + + @Override + public void punctuate(long streamTime) { + punctuated.add(streamTime); + } + + @Override + public void close() { + // do nothing + } + + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java new file mode 100644 index 0000000000000..cf0202e187554 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockSourceNode.java @@ -0,0 +1,46 @@ +/** + * 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.test; + + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.processor.internals.SourceNode; + +import java.util.ArrayList; +import java.util.concurrent.atomic.AtomicInteger; + +public class MockSourceNode extends SourceNode { + + public static final String NAME = "MOCK-SOURCE-"; + public static final AtomicInteger INDEX = new AtomicInteger(1); + + public int numReceived = 0; + public final ArrayList keys = new ArrayList<>(); + public final ArrayList values = new ArrayList<>(); + + public MockSourceNode(Deserializer keyDeserializer, Deserializer valDeserializer) { + super(NAME + INDEX.getAndIncrement(), keyDeserializer, valDeserializer); + } + + @Override + public void process(K key, V value) { + this.numReceived++; + this.keys.add(key); + this.values.add(value); + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java new file mode 100644 index 0000000000000..274e7b5bd7d4c --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockTimestampExtractor.java @@ -0,0 +1,30 @@ +/** + * 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.test; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.streams.processor.TimestampExtractor; + +/* Extract the timestamp as the offset of the record */ +public class MockTimestampExtractor implements TimestampExtractor { + + @Override + public long extract(ConsumerRecord record) { + return record.offset(); + } +} diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java new file mode 100644 index 0000000000000..75f8b4c1e4943 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -0,0 +1,317 @@ +/** + * 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.test; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.MockConsumer; +import org.apache.kafka.clients.consumer.OffsetResetStrategy; +import org.apache.kafka.clients.producer.MockProducer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.StreamingConfig; +import org.apache.kafka.streams.processor.StateStore; +import org.apache.kafka.streams.processor.TopologyBuilder; +import org.apache.kafka.streams.processor.internals.ProcessorTopology; +import org.apache.kafka.streams.processor.internals.StreamTask; +import org.apache.kafka.streams.state.KeyValueStore; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicLong; + +/** + * This class makes it easier to write tests to verify the behavior of topologies created with a {@link TopologyBuilder}. + * You can test simple topologies that have a single processor, or very complex topologies that have multiple sources, processors, + * and sinks. And because it starts with a {@link TopologyBuilder}, you can create topologies specific to your tests or you + * can use and test code you already have that uses a builder to create topologies. Best of all, the class works without a real + * Kafka broker, so the tests execute very quickly with very little overhead. + *

+ * Using the ProcessorTopologyTestDriver in tests is easy: simply instantiate the driver with a {@link StreamingConfig} and a + * TopologyBuilder, use the driver to supply an input message to the topology, and then use the driver to read and verify any + * messages output by the topology. + *

+ * Although the driver doesn't use a real Kafka broker, it does simulate Kafka {@link org.apache.kafka.clients.consumer.Consumer}s + * and {@link org.apache.kafka.clients.producer.Producer}s that read and write raw {@code byte[]} messages. You can either deal + * with messages that have {@code byte[]} keys and values, or you can supply the {@link Serializer}s and {@link Deserializer}s + * that the driver can use to convert the keys and values into objects. + * + *

Driver setup

+ *

+ * In order to create a ProcessorTopologyTestDriver instance, you need a TopologyBuilder and a {@link StreamingConfig}. The + * configuration needs to be representative of what you'd supply to the real topology, so that means including several key + * properties. For example, the following code fragment creates a configuration that specifies a local Kafka broker list + * (which is needed but not used), a timestamp extractor, and default serializers and deserializers for string keys and values: + * + *

+ * StringSerializer strSerializer = new StringSerializer();
+ * StringDeserializer strDeserializer = new StringDeserializer();
+ * Properties props = new Properties();
+ * props.setProperty(StreamingConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091");
+ * props.setProperty(StreamingConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName());
+ * props.setProperty(StreamingConfig.KEY_SERIALIZER_CLASS_CONFIG, strSerializer.getClass().getName());
+ * props.setProperty(StreamingConfig.KEY_DESERIALIZER_CLASS_CONFIG, strDeserializer.getClass().getName());
+ * props.setProperty(StreamingConfig.VALUE_SERIALIZER_CLASS_CONFIG, strSerializer.getClass().getName());
+ * props.setProperty(StreamingConfig.VALUE_DESERIALIZER_CLASS_CONFIG, strDeserializer.getClass().getName());
+ * StreamingConfig config = new StreamingConfig(props);
+ * TopologyBuilder builder = ...
+ * ProcessorTopologyTestDriver driver = new ProcessorTopologyTestDriver(config, builder);
+ * 
+ * + *

Processing messages

+ *

+ * Your test can supply new input records on any of the topics that the topology's sources consume. Here's an example of an + * input message on the topic named {@code input-topic}: + * + *

+ * driver.process("input-topic", "key1", "value1", strSerializer, strSerializer);
+ * 
+ * + * Immediately, the driver will pass the input message through to the appropriate source that consumes the named topic, + * and will invoke the processor(s) downstream of the source. If your topology's processors forward messages to sinks, + * your test can then consume these output messages to verify they match the expected outcome. For example, if our topology + * should have generated 2 messages on {@code output-topic-1} and 1 message on {@code output-topic-2}, then our test can + * obtain these messages using the {@link #readOutput(String, Deserializer, Deserializer)} method: + * + *
+ * ProducerRecord record1 = driver.readOutput("output-topic-1", strDeserializer, strDeserializer);
+ * ProducerRecord record2 = driver.readOutput("output-topic-1", strDeserializer, strDeserializer);
+ * ProducerRecord record3 = driver.readOutput("output-topic-2", strDeserializer, strDeserializer);
+ * 
+ * + * Again, our example topology generates messages with string keys and values, so we supply our string deserializer instance + * for use on both the keys and values. Your test logic can then verify whether these output records are correct. + *

+ * Finally, when completed, make sure your tests {@link #close()} the driver to release all resources and + * {@link org.apache.kafka.streams.processor.Processor}s. + * + *

Processor state

+ *

+ * Some processors use Kafka {@link StateStore state storage}, so this driver class provides the {@link #getStateStore(String)} + * and {@link #getKeyValueStore(String)} methods so that your tests can check the underlying state store(s) used by your + * topology's processors. In our previous example, after we supplied a single input message and checked the three output messages, + * our test could also check the key value store to verify the processor correctly added, removed, or updated internal state. + * Or, our test might have pre-populated some state before submitting the input message, and verified afterward that the + * processor(s) correctly updated the state. + */ +public class ProcessorTopologyTestDriver { + + private final Serializer bytesSerializer = new ByteArraySerializer(); + + private final int id; + private final ProcessorTopology topology; + private final StreamTask task; + private final MockConsumer consumer; + private final MockProducer producer; + private final MockConsumer restoreStateConsumer; + private final Map partitionsByTopic = new HashMap<>(); + private final Map offsetsByTopicPartition = new HashMap<>(); + private final Map>> outputRecordsByTopic = new HashMap<>(); + + /** + * Create a new test driver instance. + * @param config the streaming configuration for the topology + * @param builder the topology builder that will be used to create the topology instance + * @param storeNames the optional names of the state stores that are used by the topology + */ + public ProcessorTopologyTestDriver(StreamingConfig config, TopologyBuilder builder, String... storeNames) { + id = 0; + topology = builder.build(); + + // Set up the consumer and producer ... + consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + producer = new MockProducer<>(true, bytesSerializer, bytesSerializer); + restoreStateConsumer = createRestoreConsumer(id, storeNames); + + // Set up all of the topic+partition information and subscribe the consumer to each ... + for (String topic : topology.sourceTopics()) { + TopicPartition tp = new TopicPartition(topic, 1); + consumer.assign(Collections.singletonList(tp)); + partitionsByTopic.put(topic, tp); + offsetsByTopicPartition.put(tp, new AtomicLong()); + } + + task = new StreamTask(id, consumer, producer, restoreStateConsumer, partitionsByTopic.values(), topology, config); + } + + /** + * Send an input message with the given key and value on the specified topic to the topology, and then commit the messages. + * + * @param topicName the name of the topic on which the message is to be sent + * @param key the raw message key + * @param value the raw message value + */ + public void process(String topicName, byte[] key, byte[] value) { + TopicPartition tp = partitionsByTopic.get(topicName); + if (tp == null) { + throw new IllegalArgumentException("Unexpected topic: " + topicName); + } + // Add the record ... + long offset = offsetsByTopicPartition.get(tp).incrementAndGet(); + task.addRecords(tp, records(new ConsumerRecord(tp.topic(), tp.partition(), offset, key, value))); + producer.clear(); + // Process the record ... + task.process(); + task.commit(); + // Capture all the records sent to the producer ... + for (ProducerRecord record : producer.history()) { + Queue> outputRecords = outputRecordsByTopic.get(record.topic()); + if (outputRecords == null) { + outputRecords = new LinkedList<>(); + outputRecordsByTopic.put(record.topic(), outputRecords); + } + outputRecords.add(record); + } + } + + /** + * Send an input message with the given key and value on the specified topic to the topology. + * + * @param topicName the name of the topic on which the message is to be sent + * @param key the raw message key + * @param value the raw message value + * @param keySerializer the serializer for the key + * @param valueSerializer the serializer for the value + */ + public void process(String topicName, K key, V value, Serializer keySerializer, Serializer valueSerializer) { + process(topicName, keySerializer.serialize(topicName, key), valueSerializer.serialize(topicName, value)); + } + + /** + * Read the next record from the given topic. These records were output by the topology during the previous calls to + * {@link #process(String, byte[], byte[])}. + * + * @param topic the name of the topic + * @return the next record on that topic, or null if there is no record available + */ + public ProducerRecord readOutput(String topic) { + Queue> outputRecords = outputRecordsByTopic.get(topic); + if (outputRecords == null) return null; + return outputRecords.poll(); + } + + /** + * Read the next record from the given topic. These records were output by the topology during the previous calls to + * {@link #process(String, byte[], byte[])}. + * + * @param topic the name of the topic + * @param keyDeserializer the deserializer for the key type + * @param valueDeserializer the deserializer for the value type + * @return the next record on that topic, or null if there is no record available + */ + public ProducerRecord readOutput(String topic, Deserializer keyDeserializer, Deserializer valueDeserializer) { + ProducerRecord record = readOutput(topic); + if (record == null) return null; + K key = keyDeserializer.deserialize(record.topic(), record.key()); + V value = valueDeserializer.deserialize(record.topic(), record.value()); + return new ProducerRecord(record.topic(), record.partition(), key, value); + } + + private Iterable> records(ConsumerRecord record) { + return Collections.singleton(record); + } + + /** + * Get the {@link StateStore} with the given name. The name should have been supplied via + * {@link #ProcessorTopologyTestDriver(StreamingConfig, TopologyBuilder, String...) this object's constructor}, and is + * presumed to be used by a Processor within the topology. + *

+ * This is often useful in test cases to pre-populate the store before the test case instructs the topology to + * {@link #process(String, byte[], byte[]) process an input message}, and/or to check the store afterward. + * + * @param name the name of the store + * @return the state store, or null if no store has been registered with the given name + * @see #getKeyValueStore(String) + */ + public StateStore getStateStore(String name) { + return task.context().getStateStore(name); + } + + /** + * Get the {@link KeyValueStore} with the given name. The name should have been supplied via + * {@link #ProcessorTopologyTestDriver(StreamingConfig, TopologyBuilder, String...) this object's constructor}, and is + * presumed to be used by a Processor within the topology. + *

+ * This is often useful in test cases to pre-populate the store before the test case instructs the topology to + * {@link #process(String, byte[], byte[]) process an input message}, and/or to check the store afterward. + *

+ * + * @param name the name of the store + * @return the key value store, or null if no {@link KeyValueStore} has been registered with the given name + * @see #getStateStore(String) + */ + @SuppressWarnings("unchecked") + public KeyValueStore getKeyValueStore(String name) { + StateStore store = getStateStore(name); + return store instanceof KeyValueStore ? (KeyValueStore) getStateStore(name) : null; + } + + /** + * Close the driver, its topology, and all processors. + */ + public void close() { + task.close(); + } + + /** + * Utility method that creates the {@link MockConsumer} used for restoring state, which should not be done by this + * driver object unless this method is overwritten with a functional consumer. + * + * @param id the ID of the stream task + * @param storeNames the names of the stores that this + * @return the mock consumer; never null + */ + protected MockConsumer createRestoreConsumer(int id, String... storeNames) { + MockConsumer consumer = new MockConsumer(OffsetResetStrategy.LATEST) { + @Override + public synchronized void seekToEnd(TopicPartition... partitions) { + // do nothing ... + } + + @Override + public synchronized void seekToBeginning(TopicPartition... partitions) { + // do nothing ... + } + + @Override + public synchronized long position(TopicPartition partition) { + // do nothing ... + return 0L; + } + }; + // For each store name ... + for (String storeName : storeNames) { + String topicName = storeName; + // Set up the restore-state topic ... + // consumer.subscribe(new TopicPartition(topicName, 1)); + // Set up the partition that matches the ID (which is what ProcessorStateManager expects) ... + List partitionInfos = new ArrayList<>(); + partitionInfos.add(new PartitionInfo(topicName, id, null, null, null)); + consumer.updatePartitions(topicName, partitionInfos); + consumer.updateEndOffsets(Collections.singletonMap(new TopicPartition(topicName, id), 0L)); + } + return consumer; + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java new file mode 100644 index 0000000000000..b5a3b3cb9915d --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/UnlimitedWindowDef.java @@ -0,0 +1,104 @@ +/** + * 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.test; + +import org.apache.kafka.streams.kstream.Window; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.kstream.KeyValue; +import org.apache.kafka.streams.kstream.WindowDef; +import org.apache.kafka.streams.kstream.internals.FilteredIterator; +import org.apache.kafka.streams.processor.internals.Stamped; + +import java.util.Iterator; +import java.util.LinkedList; + +public class UnlimitedWindowDef implements WindowDef { + + private final String name; + + public UnlimitedWindowDef(String name) { + this.name = name; + } + + public String name() { + return name; + } + + public Window instance() { + return new UnlimitedWindow(); + } + + public class UnlimitedWindow implements Window { + + private final LinkedList>> list = new LinkedList<>(); + + @Override + public void init(ProcessorContext context) { + context.register(this, null); + } + + @Override + public Iterator find(final K key, long timestamp) { + return find(key, Long.MIN_VALUE, timestamp); + } + + @Override + public Iterator findAfter(final K key, long timestamp) { + return find(key, timestamp, Long.MAX_VALUE); + } + + @Override + public Iterator findBefore(final K key, long timestamp) { + return find(key, Long.MIN_VALUE, Long.MAX_VALUE); + } + + private Iterator find(final K key, final long startTime, final long endTime) { + return new FilteredIterator>>(list.iterator()) { + protected V filter(Stamped> item) { + if (item.value.key.equals(key) && startTime <= item.timestamp && item.timestamp <= endTime) + return item.value.value; + else + return null; + } + }; + } + + @Override + public void put(K key, V value, long timestamp) { + list.add(new Stamped<>(KeyValue.pair(key, value), timestamp)); + } + + @Override + public String name() { + return name; + } + + @Override + public void flush() { + } + + @Override + public void close() { + } + + @Override + public boolean persistent() { + return false; + } + } +}