KIP-28: Add a processor client for Kafka Streaming

This work has been contributed by Jesse Anderson, Randall Hauch, Yasuhiro Matsuda and Guozhang Wang. The detailed design can be found in https://cwiki.apache.org/confluence/display/KAFKA/KIP-28+-+Add+a+processor+client.

Author: Guozhang Wang <wangguoz@gmail.com>
Author: Yasuhiro Matsuda <yasuhiro.matsuda@gmail.com>
Author: Yasuhiro Matsuda <yasuhiro@confluent.io>
Author: ymatsuda <yasuhiro.matsuda@gmail.com>
Author: Randall Hauch <rhauch@gmail.com>
Author: Jesse Anderson <jesse@smokinghand.com>
Author: Ismael Juma <ismael@juma.me.uk>
Author: Jesse Anderson <eljefe6a@gmail.com>

Reviewers: Ismael Juma, Randall Hauch, Edward Ribeiro, Gwen Shapira, Jun Rao, Jay Kreps, Yasuhiro Matsuda, Guozhang Wang

Closes #130 from guozhangwang/streaming
This commit is contained in:
Guozhang Wang 2015-09-25 17:27:58 -07:00
parent ad120d5789
commit 263c10ab7c
104 changed files with 10083 additions and 20 deletions

View File

@ -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

View File

@ -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")
}

View File

@ -53,9 +53,13 @@
</module>
<module name="LocalVariableName"/>
<module name="LocalFinalVariableName"/>
<module name="ClassTypeParameterName"/>
<module name="MemberName"/>
<module name="MethodTypeParameterName"/>
<module name="ClassTypeParameterName">
<property name="format" value="^[A-Z0-9]*$"/>
</module>
<module name="MethodTypeParameterName">
<property name="format" value="^[A-Z0-9]*$"/>
</module>
<module name="PackageName"/>
<module name="ParameterName"/>
<module name="StaticVariableName"/>

View File

@ -90,8 +90,8 @@
</subpackage>
<subpackage name="clients">
<allow pkg="org.apache.kafka.common" />
<allow pkg="org.slf4j" />
<allow pkg="org.apache.kafka.common" />
<allow pkg="org.apache.kafka.clients" exact-match="true"/>
<allow pkg="org.apache.kafka.test" />
@ -111,6 +111,20 @@
</subpackage>
</subpackage>
<subpackage name="streams">
<allow pkg="org.apache.kafka.common"/>
<allow pkg="org.apache.kafka.test"/>
<allow pkg="org.apache.kafka.clients"/>
<allow pkg="org.apache.kafka.clients.producer" exact-match="true"/>
<allow pkg="org.apache.kafka.clients.consumer" exact-match="true"/>
<allow pkg="org.apache.kafka.streams"/>
<subpackage name="state">
<allow pkg="org.rocksdb" />
</subpackage>
</subpackage>
<subpackage name="log4jappender">
<allow pkg="org.apache.log4j" />
<allow pkg="org.apache.kafka.clients" />

View File

@ -150,11 +150,11 @@ public class ConsumerConfig extends AbstractConfig {
/** <code>key.deserializer</code> */
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 <code>Deserializer</code> interface.";
public static final String KEY_DESERIALIZER_CLASS_DOC = "Deserializer class for key that implements the <code>Deserializer</code> interface.";
/** <code>value.deserializer</code> */
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 <code>Deserializer</code> interface.";
public static final String VALUE_DESERIALIZER_CLASS_DOC = "Deserializer class for value that implements the <code>Deserializer</code> interface.";
/** <code>connections.max.idle.ms</code> */
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;

View File

@ -550,6 +550,7 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
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 @@ public class KafkaConsumer<K, V> implements Consumer<K, V> {
Deserializer.class);
this.valueDeserializer.configure(config.originals(), false);
} else {
config.ignore(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
this.valueDeserializer = valueDeserializer;
}
this.fetcher = new Fetcher<K, V>(this.client,

View File

@ -46,6 +46,7 @@ public class MockConsumer<K, V> implements Consumer<K, V> {
private final Map<String, List<PartitionInfo>> partitions;
private final SubscriptionState subscriptions;
private Map<TopicPartition, List<ConsumerRecord<K, V>>> records;
private Set<TopicPartition> paused;
private boolean closed;
private final Map<TopicPartition, Long> beginningOffsets;
private final Map<TopicPartition, Long> endOffsets;
@ -57,8 +58,9 @@ public class MockConsumer<K, V> implements Consumer<K, V> {
public MockConsumer(OffsetResetStrategy offsetResetStrategy) {
this.subscriptions = new SubscriptionState(offsetResetStrategy);
this.partitions = new HashMap<String, List<PartitionInfo>>();
this.records = new HashMap<TopicPartition, List<ConsumerRecord<K, V>>>();
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 class MockConsumer<K, V> implements Consumer<K, V> {
@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 class MockConsumer<K, V> implements Consumer<K, V> {
}
}
public Set<TopicPartition> paused() {
return Collections.unmodifiableSet(new HashSet<>(paused));
}
private void ensureNotClosed() {
if (this.closed)
throw new IllegalStateException("This consumer has already been closed.");

View File

@ -259,6 +259,7 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
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 @@ public class KafkaProducer<K, V> implements Producer<K, V> {
Serializer.class);
this.valueSerializer.configure(config.originals(), false);
} else {
config.ignore(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG);
this.valueSerializer = valueSerializer;
}
config.logUnused();

View File

@ -164,11 +164,11 @@ public class ProducerConfig extends AbstractConfig {
/** <code>key.serializer</code> */
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 <code>Serializer</code> interface.";
public static final String KEY_SERIALIZER_CLASS_DOC = "Serializer class for key that implements the <code>Serializer</code> interface.";
/** <code>value.serializer</code> */
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 <code>Serializer</code> interface.";
public static final String VALUE_SERIALIZER_CLASS_DOC = "Serializer class for value that implements the <code>Serializer</code> interface.";
/** <code>connections.max.idle.ms</code> */
public static final String CONNECTIONS_MAX_IDLE_MS_CONFIG = CommonClientConfigs.CONNECTIONS_MAX_IDLE_MS_CONFIG;

View File

@ -62,6 +62,10 @@ public class AbstractConfig {
return values.get(key);
}
public void ignore(String key) {
used.add(key);
}
public Short getShort(String key) {
return (Short) get(key);
}

View File

@ -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<Long> {
public void configure(Map<String, ?> 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
}
}

View File

@ -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<Long> {
public void configure(Map<String, ?> 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
}
}

View File

@ -25,7 +25,10 @@ import java.nio.ByteBuffer;
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 class Utils {
return existingBuffer;
}
/*
* Creates a set
* @param elems the elements
* @param <T> the type of element
* @return Set
*/
public static <T> HashSet<T> 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 <T> List<T> safe(List<T> other) {
return other == null ? Collections.<T>emptyList() : other;
}
}

View File

@ -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',
include 'core', 'contrib:hadoop-consumer', 'contrib:hadoop-producer', 'examples', 'clients', 'tools', 'streams', 'log4j-appender',
'copycat:api', 'copycat:runtime', 'copycat:json', 'copycat:file'

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.
* <p>
* 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.
* <p>
* 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.
* <p>
* 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.
* <p>
* A simple example might look like this:
* <pre>
* Map&lt;String, Object&gt; props = new HashMap&lt;&gt;();
* 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 -&gt; value.length()).to("topic2");
*
* KafkaStreaming streaming = new KafkaStreaming(builder, config);
* streaming.start();
* </pre>
*
*/
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.");
}
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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;
/** <code>state.dir</code> */
public static final String STATE_DIR_CONFIG = "state.dir";
private static final String STATE_DIR_DOC = "Directory location for state store.";
/** <code>commit.interval.ms</code> */
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.";
/** <code>poll.ms</code> */
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.";
/** <code>num.stream.threads</code> */
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.";
/** <code>buffered.records.per.partition</code> */
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.";
/** <code>state.cleanup.delay</code> */
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.";
/** <code>total.records.to.process</code> */
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.";
/** <code>window.time.ms</code> */
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.";
/** <code>timestamp.extractor</code> */
public static final String TIMESTAMP_EXTRACTOR_CLASS_CONFIG = "timestamp.extractor";
private static final String TIMESTAMP_EXTRACTOR_CLASS_DOC = "Timestamp extractor class that implements the <code>TimestampExtractor</code> interface.";
/** <code>client.id</code> */
public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG;
/** <code>key.serializer</code> */
public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
/** <code>value.serializer</code> */
public static final String VALUE_SERIALIZER_CLASS_CONFIG = ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG;
/** <code>key.deserializer</code> */
public static final String KEY_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG;
/** <code>value.deserializer</code> */
public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG;
/**
* <code>bootstrap.servers</code>
*/
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<String, Object> getConsumerConfigs() {
Map<String, Object> 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<String, Object> getProducerConfigs() {
Map<String, Object> 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());
}
}

View File

@ -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<String, String> stream1 = builder.from("topic1");
KStream<String, Integer> stream2 =
stream1.map(new KeyValueMapper<String, String, KeyValue<String, Integer>>() {
@Override
public KeyValue<String, Integer> apply(String key, String value) {
return new KeyValue<>(key, new Integer(value));
}
}).filter(new Predicate<String, Integer>() {
@Override
public boolean apply(String key, Integer value) {
return true;
}
});
KStream<String, Integer>[] streams = stream2.branch(
new Predicate<String, Integer>() {
@Override
public boolean apply(String key, Integer value) {
return (value % 2) == 0;
}
},
new Predicate<String, Integer>() {
@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();
}
}

View File

@ -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<String, String> instance() {
return new Processor<String, String>() {
private ProcessorContext context;
private KeyValueStore<String, Integer> 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<String, Integer> iter = this.kvStore.all();
while (iter.hasNext()) {
Entry<String, Integer> 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();
}
}

View File

@ -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<Object, Object> record) {
return System.currentTimeMillis();
}
}

View File

@ -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<K, V> {
/**
* Creates a new stream consists of all elements of this stream which satisfy a predicate
*
* @param predicate the instance of Predicate
* @return KStream
*/
KStream<K, V> filter(Predicate<K, V> 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<K, V> filterOut(Predicate<K, V> 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 <K1> the key type of the new stream
* @param <V1> the value type of the new stream
* @return KStream
*/
<K1, V1> KStream<K1, V1> map(KeyValueMapper<K, V, KeyValue<K1, V1>> mapper);
/**
* Creates a new stream by transforming values by a mapper to all values of this stream
*
* @param mapper the instance of ValueMapper
* @param <V1> the value type of the new stream
* @return KStream
*/
<V1> KStream<K, V1> mapValues(ValueMapper<V, V1> 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 <K1> the key type of the new stream
* @param <V1> the value type of the new stream
* @return KStream
*/
<K1, V1> KStream<K1, V1> flatMap(KeyValueMapper<K, V, Iterable<KeyValue<K1, V1>>> 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 <V1> the value type of the new stream
* @return KStream
*/
<V1> KStream<K, V1> flatMapValues(ValueMapper<V, Iterable<V1>> processor);
/**
* Creates a new windowed stream using a specified window instance.
*
* @param windowDef the instance of Window
* @return KStream
*/
KStreamWindowed<K, V> with(WindowDef<K, V> 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<K, V>[] branch(Predicate<K, V>... 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 <K1> the key type of the new stream
* @param <V1> the value type of the new stream
* @return KStream
*/
<K1, V1> KStream<K1, V1> 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 <K1> the key type of the new stream
* @param <V1> the value type of the new stream
* @return KStream
*/
<K1, V1> KStream<K1, V1> through(String topic, Serializer<K> keySerializer, Serializer<V> valSerializer, Deserializer<K1> keyDeserializer, Deserializer<V1> 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<K> keySerializer, Serializer<V> valSerializer);
/**
* Processes all elements in this stream by applying a processor.
*
* @param processorDef the class of ProcessorDef
*/
<K1, V1> KStream<K1, V1> process(ProcessorDef processorDef);
}

View File

@ -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 <K, V> KStream<K, V> 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 <K, V> KStream<K, V> from(Deserializer<? extends K> keyDeserializer, Deserializer<? extends V> valDeserializer, String... topics) {
String name = KStreamImpl.SOURCE_NAME + KStreamImpl.INDEX.getAndIncrement();
addSource(name, keyDeserializer, valDeserializer, topics);
return new KStreamImpl<>(this, name);
}
}

View File

@ -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<K, V> extends KStream<K, V> {
/**
* 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 <V1> the value type of the other stream
* @param <V2> the value type of the new stream
* @return KStream
*/
<V1, V2> KStream<K, V2> join(KStreamWindowed<K, V1> other, ValueJoiner<V, V1, V2> joiner);
}

View File

@ -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<K, V> {
public final K key;
public final V value;
public KeyValue(K key, V value) {
this.key = key;
this.value = value;
}
public static <K, V> KeyValue<K, V> pair(K key, V value) {
return new KeyValue<>(key, value);
}
}

View File

@ -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<K, V, R> {
R apply(K key, V value);
}

View File

@ -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<K, V> {
boolean apply(K key, V value);
}

View File

@ -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<K, V> implements WindowDef<K, V> {
private final String name;
private final long duration;
private final int maxCount;
private final Serializer<K> keySerializer;
private final Serializer<V> valueSerializer;
private final Deserializer<K> keyDeserializer;
private final Deserializer<V> valueDeserializer;
public SlidingWindowDef(
String name,
long duration,
int maxCount,
Serializer<K> keySerializer,
Serializer<V> valueSerializer,
Deserializer<K> keyDeseriaizer,
Deserializer<V> 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<K, V> instance() {
return new SlidingWindow();
}
public class SlidingWindow extends WindowSupport implements Window<K, V> {
private final Object lock = new Object();
private ProcessorContext context;
private int slotNum; // used as a key for Kafka log compaction
private LinkedList<K> list = new LinkedList<K>();
private HashMap<K, ValueList<V>> map = new HashMap<>();
@Override
public void init(ProcessorContext context) {
this.context = context;
RestoreFuncImpl restoreFunc = new RestoreFuncImpl();
context.register(this, restoreFunc);
for (ValueList<V> valueList : map.values()) {
valueList.clearDirtyValues();
}
this.slotNum = restoreFunc.slotNum;
}
@Override
public Iterator<V> findAfter(K key, final long timestamp) {
return find(key, timestamp, timestamp + duration);
}
@Override
public Iterator<V> findBefore(K key, final long timestamp) {
return find(key, timestamp - duration, timestamp);
}
@Override
public Iterator<V> 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<V> find(K key, final long startTime, final long endTime) {
final ValueList<V> values = map.get(key);
if (values == null) {
return Collections.emptyIterator();
} else {
return new FilteredIterator<V, Value<V>>(values.iterator()) {
@Override
protected V filter(Value<V> 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<V> 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<V> values = map.get(oldestKey);
values.removeFirst();
if (values.isEmpty()) map.remove(oldestKey);
}
}
private void evictExpired(long cutoffTime) {
while (true) {
K oldestKey = list.peekFirst();
ValueList<V> values = map.get(oldestKey);
Stamped<V> 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<K, ValueList<V>> entry : map.entrySet()) {
ValueList<V> values = entry.getValue();
if (values.hasDirtyValues()) {
K key = entry.getKey();
byte[] keyBytes = keySerializer.serialize(name, key);
Iterator<Value<V>> iterator = values.dirtyValueIterator();
while (iterator.hasNext()) {
Value<V> 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);
}
}
}
}

View File

@ -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<V1, V2, R> {
R apply(V1 value1, V2 value2);
}

View File

@ -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<V1, V2> {
V2 apply(V1 value);
}

View File

@ -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<K, V> extends StateStore {
void init(ProcessorContext context);
Iterator<V> find(K key, long timestamp);
Iterator<V> findAfter(K key, long timestamp);
Iterator<V> findBefore(K key, long timestamp);
void put(K key, V value, long timestamp);
}

View File

@ -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<K, V> {
String name();
Window<K, V> instance();
}

View File

@ -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<T, S> implements Iterator<T> {
private Iterator<S> inner;
private T nextValue = null;
public FilteredIterator(Iterator<S> 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);
}

View File

@ -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<K, V> implements ProcessorDef {
private final Predicate<K, V>[] predicates;
@SuppressWarnings("unchecked")
public KStreamBranch(Predicate... predicates) {
this.predicates = predicates;
}
@Override
public Processor instance() {
return new KStreamBranchProcessor();
}
private class KStreamBranchProcessor extends AbstractProcessor<K, V> {
@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;
}
}
}
}
}

View File

@ -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<K, V> implements ProcessorDef {
private final Predicate<K, V> predicate;
private final boolean filterOut;
public KStreamFilter(Predicate<K, V> predicate, boolean filterOut) {
this.predicate = predicate;
this.filterOut = filterOut;
}
@Override
public Processor instance() {
return new KStreamFilterProcessor();
}
private class KStreamFilterProcessor extends AbstractProcessor<K, V> {
@Override
public void process(K key, V value) {
if (filterOut ^ predicate.apply(key, value)) {
context().forward(key, value);
}
}
}
}

View File

@ -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<K1, V1, K2, V2> implements ProcessorDef {
private final KeyValueMapper<K1, V1, Iterable<KeyValue<K2, V2>>> mapper;
KStreamFlatMap(KeyValueMapper<K1, V1, Iterable<KeyValue<K2, V2>>> mapper) {
this.mapper = mapper;
}
@Override
public Processor instance() {
return new KStreamFlatMapProcessor();
}
private class KStreamFlatMapProcessor extends AbstractProcessor<K1, V1> {
@Override
public void process(K1 key, V1 value) {
for (KeyValue<K2, V2> newPair : mapper.apply(key, value)) {
context().forward(newPair.key, newPair.value);
}
}
}
}

View File

@ -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<K1, V1, V2> implements ProcessorDef {
private final ValueMapper<V1, ? extends Iterable<V2>> mapper;
KStreamFlatMapValues(ValueMapper<V1, ? extends Iterable<V2>> mapper) {
this.mapper = mapper;
}
@Override
public Processor instance() {
return new KStreamFlatMapValuesProcessor();
}
private class KStreamFlatMapValuesProcessor extends AbstractProcessor<K1, V1> {
@Override
public void process(K1 key, V1 value) {
Iterable<V2> newValues = mapper.apply(value);
for (V2 v : newValues) {
context().forward(key, v);
}
}
}
}

View File

@ -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<K, V> implements KStream<K, V> {
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<K, V> filter(Predicate<K, V> predicate) {
String name = FILTER_NAME + INDEX.getAndIncrement();
topology.addProcessor(name, new KStreamFilter<>(predicate, false), this.name);
return new KStreamImpl<>(topology, name);
}
@Override
public KStream<K, V> filterOut(final Predicate<K, V> predicate) {
String name = FILTER_NAME + INDEX.getAndIncrement();
topology.addProcessor(name, new KStreamFilter<>(predicate, true), this.name);
return new KStreamImpl<>(topology, name);
}
@Override
public <K1, V1> KStream<K1, V1> map(KeyValueMapper<K, V, KeyValue<K1, V1>> mapper) {
String name = MAP_NAME + INDEX.getAndIncrement();
topology.addProcessor(name, new KStreamMap<>(mapper), this.name);
return new KStreamImpl<>(topology, name);
}
@Override
public <V1> KStream<K, V1> mapValues(ValueMapper<V, V1> mapper) {
String name = MAPVALUES_NAME + INDEX.getAndIncrement();
topology.addProcessor(name, new KStreamMapValues<>(mapper), this.name);
return new KStreamImpl<>(topology, name);
}
@Override
public <K1, V1> KStream<K1, V1> flatMap(KeyValueMapper<K, V, Iterable<KeyValue<K1, V1>>> mapper) {
String name = FLATMAP_NAME + INDEX.getAndIncrement();
topology.addProcessor(name, new KStreamFlatMap<>(mapper), this.name);
return new KStreamImpl<>(topology, name);
}
@Override
public <V1> KStream<K, V1> flatMapValues(ValueMapper<V, Iterable<V1>> mapper) {
String name = FLATMAPVALUES_NAME + INDEX.getAndIncrement();
topology.addProcessor(name, new KStreamFlatMapValues<>(mapper), this.name);
return new KStreamImpl<>(topology, name);
}
@Override
public KStreamWindowed<K, V> with(WindowDef<K, V> 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<K, V>[] branch(Predicate<K, V>... predicates) {
String branchName = BRANCH_NAME + INDEX.getAndIncrement();
topology.addProcessor(branchName, new KStreamBranch(predicates.clone()), this.name);
KStream<K, V>[] branchChildren = (KStream<K, V>[]) 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<K, V>(), branchName);
branchChildren[i] = new KStreamImpl<>(topology, childName);
}
return branchChildren;
}
@Override
public <K1, V1> KStream<K1, V1> through(String topic,
Serializer<K> keySerializer,
Serializer<V> valSerializer,
Deserializer<K1> keyDeserializer,
Deserializer<V1> 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 <K1, V1> KStream<K1, V1> through(String topic) {
return through(topic, (Serializer<K>) null, (Serializer<V>) null, (Deserializer<K1>) null, (Deserializer<V1>) 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<K> keySerializer, Serializer<V> valSerializer) {
String name = SINK_NAME + INDEX.getAndIncrement();
topology.addSink(name, topic, keySerializer, valSerializer, this.name);
}
@Override
public <K1, V1> KStream<K1, V1> process(final ProcessorDef processorDef) {
String name = PROCESSOR_NAME + INDEX.getAndIncrement();
topology.addProcessor(name, processorDef, this.name);
return new KStreamImpl<>(topology, name);
}
}

View File

@ -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<K, V, V1, V2> implements ProcessorDef {
private static abstract class Finder<K, T> {
abstract Iterator<T> find(K key, long timestamp);
}
private final String windowName;
private final ValueJoiner<V1, V2, V> joiner;
KStreamJoin(String windowName, ValueJoiner<V1, V2, V> joiner) {
this.windowName = windowName;
this.joiner = joiner;
}
@Override
public Processor instance() {
return new KStreamJoinProcessor(windowName);
}
private class KStreamJoinProcessor extends AbstractProcessor<K, V1> {
private final String windowName;
protected Finder<K, V2> 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<K, V2> window = (Window<K, V2>) context.getStateStore(windowName);
this.finder = new Finder<K, V2>() {
Iterator<V2> find(K key, long timestamp) {
return window.find(key, timestamp);
}
};
}
@Override
public void process(K key, V1 value) {
long timestamp = context().timestamp();
Iterator<V2> iter = finder.find(key, timestamp);
if (iter != null) {
while (iter.hasNext()) {
context().forward(key, joiner.apply(value, iter.next()));
}
}
}
}
public static <T2, T1, R> ValueJoiner<T2, T1, R> reverseJoiner(final ValueJoiner<T1, T2, R> joiner) {
return new ValueJoiner<T2, T1, R>() {
@Override
public R apply(T2 value2, T1 value1) {
return joiner.apply(value1, value2);
}
};
}
}

View File

@ -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<K1, V1, K2, V2> implements ProcessorDef {
private final KeyValueMapper<K1, V1, KeyValue<K2, V2>> mapper;
public KStreamMap(KeyValueMapper<K1, V1, KeyValue<K2, V2>> mapper) {
this.mapper = mapper;
}
@Override
public Processor instance() {
return new KStreamMapProcessor();
}
private class KStreamMapProcessor extends AbstractProcessor<K1, V1> {
@Override
public void process(K1 key, V1 value) {
KeyValue<K2, V2> newPair = mapper.apply(key, value);
context().forward(newPair.key, newPair.value);
}
}
}

View File

@ -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<K1, V1, V2> implements ProcessorDef {
private final ValueMapper<V1, V2> mapper;
public KStreamMapValues(ValueMapper<V1, V2> mapper) {
this.mapper = mapper;
}
@Override
public Processor instance() {
return new KStreamMapProcessor();
}
private class KStreamMapProcessor extends AbstractProcessor<K1, V1> {
@Override
public void process(K1 key, V1 value) {
V2 newValue = mapper.apply(value);
context().forward(key, newValue);
}
}
}

View File

@ -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<K, V> implements ProcessorDef {
@Override
public Processor instance() {
return new KStreamPassThroughProcessor();
}
public class KStreamPassThroughProcessor<K, V> extends AbstractProcessor<K, V> {
@Override
public void process(K key, V value) {
context().forward(key, value);
}
}
}

View File

@ -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<K, V> implements ProcessorDef {
private final WindowDef<K, V> windowDef;
KStreamWindow(WindowDef<K, V> windowDef) {
this.windowDef = windowDef;
}
public WindowDef<K, V> window() {
return windowDef;
}
@Override
public Processor instance() {
return new KStreamWindowProcessor();
}
private class KStreamWindowProcessor extends AbstractProcessor<K, V> {
private Window<K, V> 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();
}
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<K, V> extends KStreamImpl<K, V> implements KStreamWindowed<K, V> {
private final WindowDef<K, V> windowDef;
public KStreamWindowedImpl(TopologyBuilder topology, String name, WindowDef<K, V> windowDef) {
super(topology, name);
this.windowDef = windowDef;
}
@Override
public <V1, V2> KStream<K, V2> join(KStreamWindowed<K, V1> other, ValueJoiner<V, V1, V2> valueJoiner) {
String thisWindowName = this.windowDef.name();
String otherWindowName = ((KStreamWindowedImpl<K, V1>) other).windowDef.name();
KStreamJoin<K, V2, V, V1> joinThis = new KStreamJoin<>(otherWindowName, valueJoiner);
KStreamJoin<K, V2, V1, V> joinOther = new KStreamJoin<>(thisWindowName, KStreamJoin.reverseJoiner(valueJoiner));
KStreamPassThrough<K, V2> 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);
}
}

View File

@ -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<V> {
Value<V> head = null;
Value<V> tail = null;
Value<V> dirty = null;
public void add(int slotNum, V value, long timestamp) {
Value<V> v = new Value<>(slotNum, value, timestamp);
if (tail != null) {
tail.next = v;
} else {
head = v;
}
tail = v;
if (dirty == null) dirty = v;
}
public Value<V> 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<Value<V>> iterator() {
return new ValueListIterator<V>(head);
}
public Iterator<Value<V>> dirtyValueIterator() {
return new ValueListIterator<V>(dirty);
}
}
private static class ValueListIterator<V> implements Iterator<Value<V>> {
Value<V> ptr;
ValueListIterator(Value<V> start) {
ptr = start;
}
@Override
public boolean hasNext() {
return ptr != null;
}
@Override
public Value<V> next() {
Value<V> value = ptr;
if (value != null) ptr = value.next;
return value;
}
@Override
public void remove() {
throw new UnsupportedOperationException();
}
}
public static class Value<V> extends Stamped<V> {
public final int slotNum;
private Value<V> 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> T deserialize(byte[] bytes, int offset, int length, String topic, Deserializer<T> deserializer) {
byte[] buf = new byte[length];
System.arraycopy(bytes, offset, buf, 0, length);
return deserializer.deserialize(topic, buf);
}
}

View File

@ -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 <K> the type of keys
* @param <V> the type of values
*/
public abstract class AbstractProcessor<K, V> implements Processor<K, V> {
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}.
* <p>
* This method does nothing by default; if desired, subclasses should override it with custom functionality.
* </p>
*
* @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.
* <p>
* This method does nothing by default; if desired, subclasses should override it with custom functionality.
* </p>
*/
@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;
}
}

View File

@ -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 <K> the type of keys
* @param <V> the type of values
*/
public interface Processor<K, V> {
/**
* Initialize this processor with the given context. The framework ensures this is called once per processor when the topology
* that contains it is initialized.
* <p>
* 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();
}

View File

@ -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);
<K, V> void forward(K key, V value);
<K, V> void forward(K key, V value, int childIndex);
void commit();
String topic();
int partition();
long offset();
long timestamp();
}

View File

@ -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();
}

View File

@ -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);
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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.
*
* <p>
* 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.
* </p>
*/
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();
}

View File

@ -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<Object, Object> record);
}

View File

@ -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<NodeFactory> nodeFactories = new ArrayList<>();
private Set<String> nodeNames = new HashSet<>();
private Set<String> 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<ProcessorNode> processorNodes = new ArrayList<>(nodeFactories.size());
Map<String, ProcessorNode> processorMap = new HashMap<>();
Map<String, SourceNode> 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<String> sourceTopics() {
return Collections.unmodifiableSet(sourceTopicNames);
}
}

View File

@ -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));
}
}

View File

@ -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<E> implements TimestampTracker<E> {
private final LinkedList<Stamped<E>> 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<E> elem) {
if (elem == null) throw new NullPointerException();
Stamped<E> minElem = descendingSubsequence.peekLast();
while (minElem != null && minElem.timestamp >= elem.timestamp) {
descendingSubsequence.removeLast();
minElem = descendingSubsequence.peekLast();
}
descendingSubsequence.offerLast(elem);
}
public void removeElement(Stamped<E> 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<E> stamped = descendingSubsequence.peekFirst();
if (stamped == null)
return lastKnownTime;
else
return stamped.timestamp;
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<TopicPartition, RecordQueue> partitionQueues;
private final PriorityQueue<RecordQueue> 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<TopicPartition, RecordQueue> partitionQueues, TimestampExtractor timestampExtractor) {
this.queuesByTime = new PriorityQueue<>(partitionQueues.size(), new Comparator<RecordQueue>() {
@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<ConsumerRecord<byte[], byte[]>> 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<TopicPartition> 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();
}
}

View File

@ -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<TopicPartition> partitions = this.task.partitions();
Map<Integer, List<String>> partitionsById = new HashMap<>();
int firstId = -1;
for (TopicPartition partition : partitions) {
if (!partitionsById.containsKey(partition.partition())) {
partitionsById.put(partition.partition(), new ArrayList<String>());
}
partitionsById.get(partition.partition()).add(partition.topic());
if (firstId < 0)
firstId = partition.partition();
}
List<String> topics = partitionsById.get(firstId);
for (List<String> 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 <K, V> void forward(K key, V value) {
task.forward(key, value);
}
@Override
public <K, V> 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);
}
}

View File

@ -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<K, V> {
private final List<ProcessorNode<?, ?>> children;
private final String name;
private final Processor<K, V> processor;
public ProcessorNode(String name) {
this(name, null);
}
public ProcessorNode(String name, Processor<K, V> 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<ProcessorNode<?, ?>> 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();
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<String, StateStore> stores;
private final Consumer<byte[], byte[]> restoreConsumer;
private final Map<TopicPartition, Long> restoredOffsets;
private final Map<TopicPartition, Long> checkpointedOffsets;
public ProcessorStateManager(int id, File baseDir, Consumer<byte[], byte[]> 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<byte[], byte[]> 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.<TopicPartition>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<TopicPartition, Long> ackedOffsets) throws IOException {
if (!stores.isEmpty()) {
log.debug("Closing stores.");
for (Map.Entry<String, StateStore> entry : stores.entrySet()) {
log.debug("Closing storage engine {}", entry.getKey());
entry.getValue().flush();
entry.getValue().close();
}
Map<TopicPartition, Long> 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();
}
}

View File

@ -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<ProcessorNode> processorNodes;
private final Map<String, SourceNode> sourceByTopics;
public ProcessorTopology(List<ProcessorNode> processorNodes,
Map<String, SourceNode> sourceByTopics) {
this.processorNodes = Collections.unmodifiableList(processorNodes);
this.sourceByTopics = Collections.unmodifiableMap(sourceByTopics);
}
public Set<String> sourceTopics() {
return sourceByTopics.keySet();
}
public SourceNode source(String topic) {
return sourceByTopics.get(topic);
}
public Set<SourceNode> sources() {
return new HashSet<>(sourceByTopics.values());
}
public List<ProcessorNode> processors() {
return processorNodes;
}
}

View File

@ -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<PunctuationSchedule> 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;
}
}
}

View File

@ -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<ProcessorNode> {
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);
}
}

View File

@ -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);
}

View File

@ -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<byte[], byte[]> producer;
private final Map<TopicPartition, Long> 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<byte[], byte[]> producer) {
this.producer = producer;
this.offsets = new HashMap<>();
}
public <K, V> void send(ProducerRecord<K, V> record, Serializer<K> keySerializer, Serializer<V> 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<TopicPartition, Long> offsets() {
return this.offsets;
}
}

View File

@ -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<StampedRecord> fifoQueue;
private final TimestampTracker<ConsumerRecord<Object, Object>> 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<ConsumerRecord<byte[], byte[]>> rawRecords, TimestampExtractor timestampExtractor) {
for (ConsumerRecord<byte[], byte[]> 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<Object, Object> 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;
}
}

View File

@ -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<K, V> extends ProcessorNode<K, V> {
private final String topic;
private Serializer<K> keySerializer;
private Serializer<V> valSerializer;
private ProcessorContext context;
public SinkNode(String name, String topic, Serializer<K> keySerializer, Serializer<V> 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<K>) context.keySerializer();
if (this.valSerializer == null) this.valSerializer = (Serializer<V>) 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
}
}

View File

@ -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<K, V> extends ProcessorNode<K, V> {
private Deserializer<K> keyDeserializer;
private Deserializer<V> valDeserializer;
private ProcessorContext context;
public SourceNode(String name, Deserializer<K> keyDeserializer, Deserializer<V> 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<K>) context.keyDeserializer();
if (this.valDeserializer == null) this.valDeserializer = (Deserializer<V>) context.valueDeserializer();
}
@Override
public void process(K key, V value) {
context.forward(key, value);
}
@Override
public void close() {
// do nothing
}
}

View File

@ -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<V> 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;
}
}

View File

@ -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<ConsumerRecord<Object, Object>> {
public StampedRecord(ConsumerRecord<Object, Object> 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;
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<TopicPartition, Long> 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<byte[], byte[]> consumer,
Producer<byte[], byte[]> producer,
Consumer<byte[], byte[]> restoreConsumer,
Collection<TopicPartition> 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<TopicPartition, RecordQueue> 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<TopicPartition> 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<ConsumerRecord<byte[], byte[]>> 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<TopicPartition, OffsetAndMetadata> consumedOffsetsAndMetadata = new HashMap<>(consumedOffsets.size());
for (Map.Entry<TopicPartition, Long> 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 <K, V> void forward(K key, V value) {
ProcessorNode thisNode = currNode;
for (ProcessorNode childNode : (List<ProcessorNode<K, V>>) thisNode.children()) {
currNode = childNode;
try {
childNode.process(key, value);
} finally {
currNode = thisNode;
}
}
}
@SuppressWarnings("unchecked")
public <K, V> void forward(K key, V value, int childIndex) {
ProcessorNode thisNode = currNode;
ProcessorNode childNode = (ProcessorNode<K, V>) thisNode.children().get(childIndex);
currNode = childNode;
try {
childNode.process(key, value);
} finally {
currNode = thisNode;
}
}
public ProcessorContext context() {
return processorContext;
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<byte[], byte[]> producer;
protected final Consumer<byte[], byte[]> consumer;
protected final Consumer<byte[], byte[]> restoreConsumer;
private final Map<Integer, StreamTask> 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<TopicPartition> assignment) {
addPartitions(assignment);
lastClean = time.milliseconds(); // start the cleaning cycle
}
@Override
public void onPartitionsRevoked(Collection<TopicPartition> 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<byte[], byte[]> producer,
Consumer<byte[], byte[]> consumer,
Consumer<byte[], byte[]> 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<byte[], byte[]> createProducer() {
log.info("Creating producer client for stream thread [" + this.getName() + "]");
return new KafkaProducer<>(config.getProducerConfigs(),
new ByteArraySerializer(),
new ByteArraySerializer());
}
private Consumer<byte[], byte[]> createConsumer() {
log.info("Creating consumer client for stream thread [" + this.getName() + "]");
return new KafkaConsumer<>(config.getConsumerConfigs(),
new ByteArrayDeserializer(),
new ByteArrayDeserializer());
}
private Consumer<byte[], byte[]> 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<Integer, StreamTask> 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<byte[], byte[]> 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<TopicPartition> partitionsForTask) {
metrics.taskCreationSensor.record();
return new StreamTask(id, consumer, producer, restoreConsumer, partitionsForTask, builder.build(), config);
}
private void addPartitions(Collection<TopicPartition> assignment) {
HashSet<TopicPartition> 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<TopicPartition> 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<String, String> metricTags = new LinkedHashMap<String, String>();
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()));
}
}
}

View File

@ -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<E> {
static final long NOT_KNOWN = -1L;
/**
* Adds a stamped elements to this tracker.
*
* @param elem the added element
*/
void addElement(Stamped<E> elem);
/**
* Removed a stamped elements to this tracker.
*
* @param elem the removed element
*/
void removeElement(Stamped<E> 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();
}

View File

@ -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<K, V> {
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() + ")";
}
}

View File

@ -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 <K> The key type
* @param <V> The value type
*/
public class InMemoryKeyValueStore<K, V> extends MeteredKeyValueStore<K, V> {
public InMemoryKeyValueStore(String name, ProcessorContext context) {
this(name, context, new SystemTime());
}
public InMemoryKeyValueStore(String name, ProcessorContext context, Time time) {
super(name, new MemoryStore<K, V>(name, context), context, "kafka-streams", time);
}
private static class MemoryStore<K, V> implements KeyValueStore<K, V> {
private final String name;
private final NavigableMap<K, V> 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<Entry<K, V>> entries) {
for (Entry<K, V> entry : entries)
put(entry.key(), entry.value());
}
@Override
public V delete(K key) {
return this.map.remove(key);
}
@Override
public KeyValueIterator<K, V> range(K from, K to) {
return new MemoryStoreIterator<K, V>(this.map.subMap(from, true, to, false).entrySet().iterator());
}
@Override
public KeyValueIterator<K, V> all() {
return new MemoryStoreIterator<K, V>(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<K, V> implements KeyValueIterator<K, V> {
private final Iterator<Map.Entry<K, V>> iter;
public MemoryStoreIterator(Iterator<Map.Entry<K, V>> iter) {
this.iter = iter;
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public Entry<K, V> next() {
Map.Entry<K, V> entry = iter.next();
return new Entry<>(entry.getKey(), entry.getValue());
}
@Override
public void remove() {
iter.remove();
}
@Override
public void close() {
}
}
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<K, V> extends Iterator<Entry<K, V>>, Closeable {
@Override
public void close();
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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 <K> The key type
* @param <V> The value type
*/
public interface KeyValueStore<K, V> 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<Entry<K, V>> 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<K, V> 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<K, V> all();
}

View File

@ -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<K, V> implements KeyValueStore<K, V> {
protected final KeyValueStore<K, V> 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<K> 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<K, V> 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<K>();
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<K> keyDeserializer = (Deserializer<K>) context.keyDeserializer();
final Deserializer<V> valDeserializer = (Deserializer<V>) 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<Entry<K, V>> entries) {
long startNs = time.nanoseconds();
try {
this.inner.putAll(entries);
for (Entry<K, V> 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<K, V> range(K from, K to) {
return new MeteredKeyValueIterator<K, V>(this.inner.range(from, to), this.rangeTime);
}
@Override
public KeyValueIterator<K, V> all() {
return new MeteredKeyValueIterator<K, V>(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<K> keySerializer = (Serializer<K>) context.keySerializer();
Serializer<V> valueSerializer = (Serializer<V>) 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<K1, V1> implements KeyValueIterator<K1, V1> {
private final KeyValueIterator<K1, V1> iter;
private final Sensor sensor;
private final long startNs;
public MeteredKeyValueIterator(KeyValueIterator<K1, V1> iter, Sensor sensor) {
this.iter = iter;
this.sensor = sensor;
this.startNs = time.nanoseconds();
}
@Override
public boolean hasNext() {
return iter.hasNext();
}
@Override
public Entry<K1, V1> 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());
}
}
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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=&gt;offsets to a file. The format of the file is UTF-8 text containing the following:
* <pre>
* &lt;version&gt;
* &lt;n&gt;
* &lt;topic_name_1&gt; &lt;partition_1&gt; &lt;offset_1&gt;
* .
* .
* .
* &lt;topic_name_n&gt; &lt;partition_n&gt; &lt;offset_n&gt;
* </pre>
* 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<TopicPartition, Long> 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<TopicPartition, Long> 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<TopicPartition, Long> 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<TopicPartition, Long> offsets = new HashMap<TopicPartition, Long>();
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();
}
}

View File

@ -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<byte[], byte[]> {
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<byte[], byte[]> {
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<Entry<byte[], byte[]>> entries) {
for (Entry<byte[], byte[]> 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<byte[], byte[]> range(byte[] from, byte[] to) {
return new RocksDBRangeIterator(db.newIterator(), from, to);
}
@Override
public KeyValueIterator<byte[], byte[]> 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<byte[], byte[]> {
private final RocksIterator iter;
public RocksDbIterator(RocksIterator iter) {
this.iter = iter;
}
protected byte[] peekKey() {
return this.getEntry().key();
}
protected Entry<byte[], byte[]> getEntry() {
return new Entry<>(iter.key(), iter.value());
}
@Override
public boolean hasNext() {
return iter.isValid();
}
@Override
public Entry<byte[], byte[]> next() {
if (!hasNext())
throw new NoSuchElementException();
Entry<byte[], byte[]> 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<byte[]> {
@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<byte[]> 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;
}
}
}
}

View File

@ -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");
}
}

View File

@ -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<Integer> list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5);
Iterator<String> filtered = new FilteredIterator<String, Integer>(list.iterator()) {
protected String filter(Integer i) {
if (i % 3 == 0) return i.toString();
return null;
}
};
List<String> expected = Arrays.asList("3", "9", "6", "3");
List<String> result = new ArrayList<String>();
while (filtered.hasNext()) {
result.add(filtered.next());
}
assertEquals(expected, result);
}
@Test
public void testEmptySource() {
List<Integer> list = new ArrayList<Integer>();
Iterator<String> filtered = new FilteredIterator<String, Integer>(list.iterator()) {
protected String filter(Integer i) {
if (i % 3 == 0) return i.toString();
return null;
}
};
List<String> expected = new ArrayList<String>();
List<String> result = new ArrayList<String>();
while (filtered.hasNext()) {
result.add(filtered.next());
}
assertEquals(expected, result);
}
@Test
public void testNoMatch() {
List<Integer> list = Arrays.asList(3, 1, 4, 1, 5, 9, 2, 6, 5, 3, 5);
Iterator<String> filtered = new FilteredIterator<String, Integer>(list.iterator()) {
protected String filter(Integer i) {
if (i % 7 == 0) return i.toString();
return null;
}
};
List<String> expected = new ArrayList<String>();
List<String> result = new ArrayList<String>();
while (filtered.hasNext()) {
result.add(filtered.next());
}
assertEquals(expected, result);
}
}

View File

@ -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<Integer, String> isEven = new Predicate<Integer, String>() {
@Override
public boolean apply(Integer key, String value) {
return (key % 2) == 0;
}
};
Predicate<Integer, String> isMultipleOfThree = new Predicate<Integer, String>() {
@Override
public boolean apply(Integer key, String value) {
return (key % 3) == 0;
}
};
Predicate<Integer, String> isOdd = new Predicate<Integer, String>() {
@Override
public boolean apply(Integer key, String value) {
return (key % 2) != 0;
}
};
final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6};
KStream<Integer, String> stream;
KStream<Integer, String>[] branches;
MockProcessorDef<Integer, String>[] processors;
stream = builder.from(keyDeserializer, valDeserializer, topicName);
branches = stream.branch(isEven, isMultipleOfThree, isOdd);
assertEquals(3, branches.length);
processors = (MockProcessorDef<Integer, String>[]) 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());
}
}

View File

@ -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<Integer, String> isMultipleOfThree = new Predicate<Integer, String>() {
@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<Integer, String> stream;
MockProcessorDef<Integer, String> 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<Integer, String> stream;
MockProcessorDef<Integer, String> 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());
}
}

View File

@ -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<Integer, String, Iterable<KeyValue<String, String>>> mapper =
new KeyValueMapper<Integer, String, Iterable<KeyValue<String, String>>>() {
@Override
public Iterable<KeyValue<String, String>> apply(Integer key, String value) {
ArrayList<KeyValue<String, String>> 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<Integer, String> stream;
MockProcessorDef<String, String> 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));
}
}
}

View File

@ -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<String, Iterable<String>> mapper =
new ValueMapper<String, Iterable<String>>() {
@Override
public Iterable<String> apply(String value) {
ArrayList<String> result = new ArrayList<String>();
result.add(value.toLowerCase());
result.add(value);
return result;
}
};
final int[] expectedKeys = {0, 1, 2, 3};
KStream<Integer, String> stream;
MockProcessorDef<Integer, String> 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));
}
}
}

View File

@ -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<String> deserializer = new StringDeserializer();
final KStreamBuilder builder = new KStreamBuilder();
KStream<String, String> source1 = builder.from(deserializer, deserializer, "topic-1", "topic-2");
KStream<String, String> source2 = builder.from(deserializer, deserializer, "topic-3", "topic-4");
KStream<String, String> stream1 =
source1.filter(new Predicate<String, String>() {
@Override
public boolean apply(String key, String value) {
return true;
}
}).filterOut(new Predicate<String, String>() {
@Override
public boolean apply(String key, String value) {
return false;
}
});
KStream<String, Integer> stream2 = stream1.mapValues(new ValueMapper<String, Integer>() {
@Override
public Integer apply(String value) {
return new Integer(value);
}
});
KStream<String, Integer> stream3 = source2.flatMapValues(new ValueMapper<String, Iterable<Integer>>() {
@Override
public Iterable<Integer> apply(String value) {
return Collections.singletonList(new Integer(value));
}
});
KStream<String, Integer>[] streams2 = stream2.branch(
new Predicate<String, Integer>() {
@Override
public boolean apply(String key, Integer value) {
return (value % 2) == 0;
}
},
new Predicate<String, Integer>() {
@Override
public boolean apply(String key, Integer value) {
return true;
}
}
);
KStream<String, Integer>[] streams3 = stream3.branch(
new Predicate<String, Integer>() {
@Override
public boolean apply(String key, Integer value) {
return (value % 2) == 0;
}
},
new Predicate<String, Integer>() {
@Override
public boolean apply(String key, Integer value) {
return true;
}
}
);
KStream<String, Integer> stream4 = streams2[0].with(new UnlimitedWindowDef<String, Integer>("window"))
.join(streams3[0].with(new UnlimitedWindowDef<String, Integer>("window")), new ValueJoiner<Integer, Integer, Integer>() {
@Override
public Integer apply(Integer value1, Integer value2) {
return value1 + value2;
}
});
KStream<String, Integer> stream5 = streams2[1].with(new UnlimitedWindowDef<String, Integer>("window"))
.join(streams3[1].with(new UnlimitedWindowDef<String, Integer>("window")), new ValueJoiner<Integer, Integer, Integer>() {
@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());
}
}

View File

@ -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<String, String, String> joiner = new ValueJoiner<String, String, String>() {
@Override
public String apply(String value1, String value2) {
return value1 + "+" + value2;
}
};
private ValueMapper<String, String> valueMapper = new ValueMapper<String, String>() {
@Override
public String apply(String value) {
return "#" + value;
}
};
private ValueMapper<String, Iterable<String>> valueMapper2 = new ValueMapper<String, Iterable<String>>() {
@Override
public Iterable<String> apply(String value) {
return (Iterable<String>) Utils.mkSet(value);
}
};
private KeyValueMapper<Integer, String, KeyValue<Integer, String>> keyValueMapper =
new KeyValueMapper<Integer, String, KeyValue<Integer, String>>() {
@Override
public KeyValue<Integer, String> apply(Integer key, String value) {
return KeyValue.pair(key, value);
}
};
KeyValueMapper<Integer, String, KeyValue<Integer, Iterable<String>>> keyValueMapper2 =
new KeyValueMapper<Integer, String, KeyValue<Integer, Iterable<String>>>() {
@Override
public KeyValue<Integer, Iterable<String>> apply(Integer key, String value) {
return KeyValue.pair(key, (Iterable<String>) Utils.mkSet(value));
}
};
@Test
public void testJoin() {
KStreamBuilder builder = new KStreamBuilder();
final int[] expectedKeys = new int[]{0, 1, 2, 3};
KStream<Integer, String> stream1;
KStream<Integer, String> stream2;
KStreamWindowed<Integer, String> windowed1;
KStreamWindowed<Integer, String> windowed2;
MockProcessorDef<Integer, String> processor;
String[] expected;
processor = new MockProcessorDef<>();
stream1 = builder.from(keyDeserializer, valDeserializer, topic1);
stream2 = builder.from(keyDeserializer, valDeserializer, topic2);
windowed1 = stream1.with(new UnlimitedWindowDef<Integer, String>("window1"));
windowed2 = stream2.with(new UnlimitedWindowDef<Integer, String>("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
}

View File

@ -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<Integer, String, KeyValue<String, Integer>> mapper =
new KeyValueMapper<Integer, String, KeyValue<String, Integer>>() {
@Override
public KeyValue<String, Integer> apply(Integer key, String value) {
return KeyValue.pair(value, key);
}
};
final int[] expectedKeys = new int[]{0, 1, 2, 3};
KStream<Integer, String> stream;
MockProcessorDef<String, Integer> 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));
}
}
}

View File

@ -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<String, Integer> mapper =
new ValueMapper<String, Integer>() {
@Override
public Integer apply(String value) {
return value.length();
}
};
final int[] expectedKeys = {1, 10, 100, 1000};
KStream<Integer, String> stream;
MockProcessorDef<Integer, Integer> 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));
}
}
}

View File

@ -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<Integer, String> stream;
WindowDef<Integer, String> windowDef;
windowDef = new UnlimitedWindowDef<>(windowName);
stream = builder.from(keyDeserializer, valDeserializer, topicName);
stream.with(windowDef);
KStreamTestDriver driver = new KStreamTestDriver(builder);
Window<Integer, String> window = (Window<Integer, String>) 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 <T> int countItem(Iterator<T> iter) {
int i = 0;
while (iter.hasNext()) {
i++;
iter.next();
}
return i;
}
}

View File

@ -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);
}
}

View File

@ -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<String> elem(long timestamp) {
return new Stamped<>("", timestamp);
}
@SuppressWarnings("unchecked")
@Test
public void testTracking() {
TimestampTracker<String> 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<String>) elems[insertionIndex++]);
assertEquals(100L, tracker.get());
// add 101
tracker.addElement((Stamped<String>) elems[insertionIndex++]);
assertEquals(100L, tracker.get());
// remove 100
tracker.removeElement((Stamped<String>) elems[removalIndex++]);
assertEquals(101L, tracker.get());
// add 102
tracker.addElement((Stamped<String>) elems[insertionIndex++]);
assertEquals(101L, tracker.get());
// add 98
tracker.addElement((Stamped<String>) elems[insertionIndex++]);
assertEquals(98L, tracker.get());
// add 99
tracker.addElement((Stamped<String>) elems[insertionIndex++]);
assertEquals(98L, tracker.get());
// add 100
tracker.addElement((Stamped<String>) elems[insertionIndex++]);
assertEquals(98L, tracker.get());
// remove 101
tracker.removeElement((Stamped<String>) elems[removalIndex++]);
assertEquals(98L, tracker.get());
// remove 102
tracker.removeElement((Stamped<String>) elems[removalIndex++]);
assertEquals(98L, tracker.get());
// remove 98
tracker.removeElement((Stamped<String>) elems[removalIndex++]);
assertEquals(99L, tracker.get());
// remove 99
tracker.removeElement((Stamped<String>) elems[removalIndex++]);
assertEquals(100L, tracker.get());
// remove 100
tracker.removeElement((Stamped<String>) elems[removalIndex++]);
assertEquals(100L, tracker.get());
assertEquals(insertionIndex, removalIndex);
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<Integer> intSerializer = new IntegerSerializer();
private final Deserializer<Integer> 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<TopicPartition, RecordQueue>() {
{
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<ConsumerRecord<byte[], byte[]>> 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<ConsumerRecord<byte[], byte[]>> 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());
}
}

View File

@ -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<Integer> 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<Integer> deserializer = new IntegerDeserializer();
@Override
public void apply(byte[] key, byte[] value) {
keys.add(deserializer.deserialize("", key));
}
};
}
private class MockRestoreConsumer extends MockConsumer<byte[], byte[]> {
private final Serializer<Integer> 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<ConsumerRecord<byte[], byte[]>> 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<Integer, Integer> 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<TopicPartition> 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<byte[], byte[]> poll(long timeout) {
// add buffered records to MockConsumer
for (ConsumerRecord<byte[], byte[]> record : recordBuffer) {
super.addRecord(record);
}
recordBuffer.clear();
ConsumerRecords<byte[], byte[]> records = super.poll(timeout);
// set the current offset
Iterable<ConsumerRecord<byte[], byte[]>> partitionRecords = records.records(assignedPartition);
for (ConsumerRecord<byte[], byte[]> 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.<TopicPartition, Long>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.<TopicPartition, Long>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<Integer> 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.<TopicPartition, Long>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<Integer> 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.<TopicPartition, Long>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.<TopicPartition, Long>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.<TopicPartition, Long>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<TopicPartition, Long> 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<TopicPartition, Long> checkpointedOffsets = newCheckpoint.read();
assertEquals(1, checkpointedOffsets.size());
assertEquals(new Long(123L + 1L), checkpointedOffsets.get(new TopicPartition("persistentStore", 1)));
} finally {
Utils.delete(baseDir);
}
}
}

View File

@ -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> STRING_SERIALIZER = new StringSerializer();
private static final Deserializer<String> 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<Path>() {
@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<String, String> 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<String, String> 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<String, String> {
@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<String, String> {
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<String, String> {
private KeyValueStore<String, String> 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<String, String> 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<Object, Object> record) {
return timestamp;
}
}
}

View File

@ -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<String, String> 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<String, String> {
public final ArrayList<Long> 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() {
}
}
}

View File

@ -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
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* 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<Integer> intSerializer = new IntegerSerializer();
private final Deserializer<Integer> 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<ConsumerRecord<byte[], byte[]>> 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<ConsumerRecord<byte[], byte[]>> 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<ConsumerRecord<byte[], byte[]>> 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());
}
}

View File

@ -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<Integer> intSerializer = new IntegerSerializer();
private final Deserializer<Integer> intDeserializer = new IntegerDeserializer();
private final Serializer<byte[]> bytesSerializer = new ByteArraySerializer();
private final TopicPartition partition1 = new TopicPartition("topic1", 1);
private final TopicPartition partition2 = new TopicPartition("topic2", 1);
private final HashSet<TopicPartition> 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<String, SourceNode>() {
{
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<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
private final MockProducer<byte[], byte[]> producer = new MockProducer<>(false, bytesSerializer, bytesSerializer);
private final MockConsumer<byte[], byte[]> 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<ConsumerRecord<byte[], byte[]>> records(ConsumerRecord<byte[], byte[]>... recs) {
return Arrays.asList(recs);
}
}

View File

@ -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<byte[], byte[]> consumer,
Producer<byte[], byte[]> producer,
Consumer<byte[], byte[]> restoreConsumer,
Collection<TopicPartition> 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<byte[], byte[]> producer = new MockProducer<>(true, serializer, serializer);
MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
final MockConsumer<byte[], byte[]> 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<TopicPartition> partitionsForTask) {
return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config);
}
};
ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener;
assertTrue(thread.tasks().isEmpty());
List<TopicPartition> revokedPartitions;
List<TopicPartition> assignedPartitions;
Set<TopicPartition> expectedGroup1;
Set<TopicPartition> 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<byte[], byte[]> producer = new MockProducer<>(true, serializer, serializer);
MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
final MockConsumer<byte[], byte[]> 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<TopicPartition> 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<TopicPartition> revokedPartitions;
List<TopicPartition> assignedPartitions;
Map<Integer, StreamTask> 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<byte[], byte[]> producer = new MockProducer<>(true, serializer, serializer);
MockConsumer<byte[], byte[]> consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST);
final MockConsumer<byte[], byte[]> 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<TopicPartition> partitionsForTask) {
return new TestStreamTask(id, consumer, producer, mockRestoreConsumer, partitionsForTask, builder.build(), config);
}
};
ConsumerRebalanceListener rebalanceListener = thread.rebalanceListener;
List<TopicPartition> revokedPartitions;
List<TopicPartition> 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);
}
}
}

View File

@ -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 <K, V> void forward(K key, V value) {
ProcessorNode thisNode = currNode;
for (ProcessorNode childNode : (List<ProcessorNode<K, V>>) thisNode.children()) {
currNode = childNode;
try {
childNode.process(key, value);
} finally {
currNode = thisNode;
}
}
}
@SuppressWarnings("unchecked")
public <K, V> void forward(K key, V value, int childIndex) {
ProcessorNode thisNode = currNode;
ProcessorNode childNode = (ProcessorNode<K, V>) thisNode.children().get(childIndex);
currNode = childNode;
try {
childNode.process(key, value);
} finally {
currNode = thisNode;
}
}
}

View File

@ -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<String, StateStore> 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 <K, V> void forward(K key, V value) {
driver.forward(key, value);
}
@Override
@SuppressWarnings("unchecked")
public <K, V> 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;
}
}

View File

@ -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<K, V> implements ProcessorDef {
public final ArrayList<String> processed = new ArrayList<>();
public final ArrayList<Long> punctuated = new ArrayList<>();
public Processor instance() {
return new MockProcessor();
}
public class MockProcessor implements Processor<K, V> {
@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
}
}
}

Some files were not shown because too many files have changed in this diff Show More